You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by sa...@apache.org on 2019/05/29 23:08:41 UTC

[drill] 05/05: DRILL-4782 / DRILL-7139: Fix DATE_ADD and TO_TIME functions

This is an automated email from the ASF dual-hosted git repository.

sachouche pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 2615d68de4e44b1f03f5c047018548c06a7396b4
Author: Dmytriy Grinchenko <dm...@gmail.com>
AuthorDate: Mon May 6 09:01:29 2019 +0300

    DRILL-4782 / DRILL-7139: Fix DATE_ADD and TO_TIME functions
    
    - cast function for the day interval changed to round milliseconds to complete days
    - ToDateTypeFunctions#toTime now returning milliseconds of day
    - updated the way how DayInterval subtracts and adds, to follow the cast function logic
    
    UT core updates:
    
    - added vectorValue function to the queryBuilder to simplify retrieving value of the vector
    - refactored singleton query result functions at queryBuilder
---
 .../templates/CastStringTypesToInterval.java       |  29 ++--
 .../IntervalIntervalArithmetic.java                |  22 ++-
 .../ToDateTypeFunctions.java                       |   2 +-
 .../expr/fn/impl/TestIntervalDayFunctions.java     |  95 +++++++++++++
 .../drill/exec/fn/impl/TestCastFunctions.java      |  69 +++++----
 .../exec/fn/impl/testing/TestDateConversions.java  |  12 ++
 .../java/org/apache/drill/test/QueryBuilder.java   | 154 +++++++++++++++------
 7 files changed, 304 insertions(+), 79 deletions(-)

diff --git a/exec/java-exec/src/main/codegen/templates/CastStringTypesToInterval.java b/exec/java-exec/src/main/codegen/templates/CastStringTypesToInterval.java
index 196e860..7acfd69 100644
--- a/exec/java-exec/src/main/codegen/templates/CastStringTypesToInterval.java
+++ b/exec/java-exec/src/main/codegen/templates/CastStringTypesToInterval.java
@@ -86,24 +86,33 @@ public class CastEmptyString${type.from}To${type.to} implements DrillSimpleFunc
     org.joda.time.Period period = org.joda.time.Period.parse(input);
 
     <#if type.to == "Interval" || type.to == "NullableInterval">
-    out.months       = (period.getYears() * org.apache.drill.exec.vector.DateUtilities.yearsToMonths) + period.getMonths();
+    out.months       = period.getYears() * org.apache.drill.exec.vector.DateUtilities.yearsToMonths + period.getMonths();
 
     out.days         = period.getDays();
 
-    out.milliseconds = (period.getHours() * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
-                       (period.getMinutes() * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
-                       (period.getSeconds() * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
-                       (period.getMillis());
+    out.milliseconds = period.getHours() * org.apache.drill.exec.vector.DateUtilities.hoursToMillis +
+                       period.getMinutes() * org.apache.drill.exec.vector.DateUtilities.minutesToMillis +
+                       period.getSeconds() * org.apache.drill.exec.vector.DateUtilities.secondsToMillis +
+                       period.getMillis();
 
     <#elseif type.to == "IntervalDay" || type.to == "NullableIntervalDay">
     out.days         = period.getDays();
 
-    out.milliseconds = (period.getHours() * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
-                       (period.getMinutes() * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
-                       (period.getSeconds() * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
-                       (period.getMillis());
+    long millis = period.getHours() * (long) org.apache.drill.exec.vector.DateUtilities.hoursToMillis +
+                  period.getMinutes() * (long) org.apache.drill.exec.vector.DateUtilities.minutesToMillis +
+                  period.getSeconds() * (long) org.apache.drill.exec.vector.DateUtilities.secondsToMillis +
+                  period.getMillis();
+
+    if (millis >= org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis) {
+      int daysFromMillis = (int) (millis / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
+      millis -= daysFromMillis * org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
+      out.days += daysFromMillis;
+    }
+
+    out.milliseconds = (int) millis;
+
     <#elseif type.to == "IntervalYear" || type.to == "NullableIntervalYear">
-    out.value = (period.getYears() * org.apache.drill.exec.vector.DateUtilities.yearsToMonths) + period.getMonths();
+    out.value = period.getYears() * org.apache.drill.exec.vector.DateUtilities.yearsToMonths + period.getMonths();
     </#if>
   }
 }
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java
index 7fecabb..ccd8e1f 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java
@@ -65,7 +65,15 @@ public class ${intervaltype}Functions {
             out.value = left.value + right.value;
             <#elseif intervaltype == "IntervalDay">
             out.days = left.days + right.days;
-            out.milliseconds = left.milliseconds + right.milliseconds;
+            long millis = (long) left.milliseconds + right.milliseconds;
+
+            if (millis >= org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis) {
+              int daysFromMillis = (int) (millis / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
+
+              millis -= daysFromMillis * org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
+              out.days += daysFromMillis;
+            }
+            out.milliseconds = (int) millis;
             </#if>
         }
     }
@@ -90,6 +98,18 @@ public class ${intervaltype}Functions {
             <#elseif intervaltype == "IntervalDay">
             out.days = left.days - right.days;
             out.milliseconds = left.milliseconds - right.milliseconds;
+
+            int daysFromMillis = out.milliseconds/org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
+            if (daysFromMillis != 0) {
+              out.milliseconds -= org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis*daysFromMillis;
+              out.days -= Math.abs(daysFromMillis);
+            }
+
+            // if milliseconds are bellow zero, substract them from the days
+            if (out.milliseconds < 0 && out.days > 0) {
+              out.days  -= 1;
+              out.milliseconds =  org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis + out.milliseconds;
+            }
             </#if>
         }
     }
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java
index 6ef196f..7f5784a 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java
@@ -82,7 +82,7 @@ public class G${convert.from}To${convert.to} implements DrillSimpleFunc {
     <#elseif convert.to == "TimeStamp" || convert.to == "NullableTimeStamp">
     out.value = org.joda.time.DateTime.parse(input, format).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis();
     <#elseif convert.to == "Time" || convert.to == "NullableTime">
-    out.value = (int) ((format.parseDateTime(input)).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
+    out.value = format.parseDateTime(input).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillisOfDay();
     </#if>
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestIntervalDayFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestIntervalDayFunctions.java
new file mode 100644
index 0000000..2bc29c5
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestIntervalDayFunctions.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.expr.fn.impl;
+
+import org.apache.drill.exec.vector.IntervalDayVector;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestIntervalDayFunctions extends ClusterTest {
+  @BeforeClass
+  public static void setUp() throws Exception {
+    startCluster(ClusterFixture.builder(dirTestWatcher));
+  }
+
+  @Test
+  public void testIntervalDaySubtractFunction() throws Exception {
+    QueryBuilder.VectorQueryReader<String,IntervalDayVector> vectorQueryReader =
+        (recordsCount, vector) -> vector.getAccessor().getAsStringBuilder(0).toString();
+
+    String result = queryBuilder()
+        .sql("select cast('P6D' as interval day) - cast('P5DT4M20S' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("0 days 23:55:40", result);
+
+    result = queryBuilder()
+        .sql("select cast('P4D' as interval day) - cast('P4DT4M' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("0 days 0:-4:00", result);
+
+    result = queryBuilder()
+        .sql("select cast('P4D' as interval day) - cast('PT4M' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("3 days 23:56:00", result);
+
+    result = queryBuilder()
+        .sql("select cast('P4D' as interval day) - cast('P5D' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("-1 day 0:00:00", result);
+
+    result = queryBuilder()
+        .sql("select cast('P4D' as interval day) - cast('P4DT23H59M59S' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("0 days -23:-59:-59", result);
+
+    result = queryBuilder()
+        .sql("select cast('P4D' as interval day) - cast('P5DT23H59S' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("-1 day -23:00:-59", result);
+
+    result = queryBuilder()
+        .sql("select cast('P2D' as interval day) - cast('P1DT4M' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("0 days 23:56:00", result);
+  }
+
+  @Test
+  public void testIntervalDayPlusFunction() throws Exception {
+    QueryBuilder.VectorQueryReader<String,IntervalDayVector> vectorQueryReader =
+        (recordsCount, vector) -> vector.getAccessor().getAsStringBuilder(0).toString();
+
+    String result = queryBuilder()
+        .sql("select cast('P1D' as interval day) + cast('P2DT1H' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("3 days 1:00:00", result);
+
+    result = queryBuilder()
+        .sql("select cast('PT12H' as interval day) + cast('PT12H' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("1 day 0:00:00", result);
+
+    result = queryBuilder()
+        .sql("select cast('PT11H' as interval day) + cast('PT12H59M60S' as interval day) as i")
+        .vectorValue("i", IntervalDayVector.class, vectorQueryReader);
+    Assert.assertEquals("1 day 0:00:00", result);
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
index 73b4b94..fc06cb1 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
@@ -30,15 +30,13 @@ import java.util.Set;
 import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.common.exceptions.UserRemoteException;
-import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
-import org.apache.drill.exec.rpc.user.QueryDataBatch;
+import org.apache.drill.exec.vector.IntervalDayVector;
 import org.apache.drill.exec.vector.IntervalYearVector;
 import org.apache.drill.test.ClusterFixture;
 import org.apache.drill.test.ClusterTest;
@@ -698,30 +696,53 @@ public class TestCastFunctions extends ClusterTest {
 
   @Test // DRILL-6783
   public void testCastVarCharIntervalYear() throws Exception {
-    String query = "select cast('P31M' as interval month) as i from cp.`employee.json` limit 10";
-    List<QueryDataBatch> result = queryBuilder().sql(query).results();
-    RecordBatchLoader loader = new RecordBatchLoader(cluster.drillbit().getContext().getAllocator());
-
-    QueryDataBatch b = result.get(0);
-    loader.load(b.getHeader().getDef(), b.getData());
-
-    IntervalYearVector vector = (IntervalYearVector) loader.getValueAccessorById(
-          IntervalYearVector.class,
-          loader.getValueVectorId(SchemaPath.getCompoundPath("i")).getFieldIds())
-        .getValueVector();
-
-    Set<String> resultSet = new HashSet<>();
-    for (int i = 0; i < loader.getRecordCount(); i++) {
-      String displayValue = vector.getAccessor().getAsStringBuilder(i).toString();
-      resultSet.add(displayValue);
-    }
+    Set<String> results = queryBuilder()
+        .sql("select cast('P31M' as interval month) as i from cp.`employee.json` limit 10")
+        .vectorValue(
+            "i",
+            IntervalYearVector.class,
+            (recordCount, vector) -> {
+              Set<String> r = new HashSet<>();
+              for (int i = 0; i < recordCount; i++) {
+                r.add(vector.getAccessor().getAsStringBuilder(i).toString());
+              }
+              return r;
+            }
+        );
 
     Assert.assertEquals(
-        "Casting literal string as INTERVAL should yield the same result for each row", 1, resultSet.size());
-    Assert.assertThat(resultSet, hasItem("2 years 7 months"));
+        "Casting literal string as INTERVAL should yield the same result for each row", 1, results.size());
+    Assert.assertThat(results, hasItem("2 years 7 months"));
+  }
 
-    b.release();
-    loader.clear();
+  @Test
+  public void testCastVarCharIntervalDay() throws Exception {
+    String result = queryBuilder()
+        .sql("select cast('PT1H' as interval minute) as i from (values(1))")
+        .vectorValue(
+            "i",
+            IntervalDayVector.class,
+            (recordsCount, vector) -> vector.getAccessor().getAsStringBuilder(0).toString()
+        );
+    Assert.assertEquals(result, "0 days 1:00:00");
+
+    result = queryBuilder()
+        .sql("select cast(concat('PT',107374,'M') as interval minute) as i from (values(1))")
+        .vectorValue(
+            "i",
+            IntervalDayVector.class,
+            (recordsCount, vector) -> vector.getAccessor().getAsStringBuilder(0).toString()
+        );
+    Assert.assertEquals(result, "74 days 13:34:00");
+
+    result = queryBuilder()
+        .sql("select cast(concat('PT',107375,'M') as interval minute) as i from (values(1))")
+        .vectorValue(
+            "i",
+            IntervalDayVector.class,
+            (recordsCount, vector) -> vector.getAccessor().getAsStringBuilder(0).toString()
+        );
+    Assert.assertEquals(result, "74 days 13:35:00");
   }
 
   @Test // DRILL-6959
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
index 363d326..c820408 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
@@ -108,6 +108,18 @@ public class TestDateConversions extends BaseTestQuery {
   }
 
   @Test
+  public void testToTimeWithDateTimePatternFormat() throws Exception {
+    mockUsDateFormatSymbols();
+
+    testBuilder()
+        .sqlQuery("select TO_TIME('2016-03-03 00:00', 'yyyy-MM-dd HH:mm') as `result` from (values(1))")
+        .unOrdered()
+        .baselineColumns("result")
+        .baselineValues(LocalTime.of(0,0,0))
+        .go();
+  }
+
+  @Test
   public void testPostgresTime() throws Exception {
     mockUsDateFormatSymbols();
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
index 30bff62..795cdae 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
@@ -51,6 +51,7 @@ import org.apache.drill.exec.rpc.user.UserResultsListener;
 import org.apache.drill.exec.util.VectorUtil;
 import org.apache.drill.exec.vector.NullableVarCharVector;
 import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ScalarReader;
 import org.apache.drill.test.BufferingQueryEventListener.QueryEvent;
 import org.apache.drill.test.ClientFixture.StatementParser;
 import org.apache.drill.test.rowSet.DirectRowSet;
@@ -58,6 +59,7 @@ import org.apache.drill.test.rowSet.RowSet;
 import org.apache.drill.test.rowSet.RowSetReader;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.joda.time.Period;
 
 /**
  * Builder for a Drill query. Provides all types of query formats,
@@ -213,6 +215,23 @@ public class QueryBuilder {
     public QueryState finalState() { return finalState; }
   }
 
+  /**
+   * Scalar reader function interface for a set of reader methods
+   * @param <T> - reader returned type
+   */
+  private interface SingletonScalarReader<T> {
+    T read(ScalarReader reader);
+  }
+
+  /**
+   * VectorQueryReader function interface
+   * @param <V> - vector class
+   * @param <T> - result type
+   */
+  public interface VectorQueryReader<T, V> {
+    T read(int recordsCount, V vector);
+  }
+
   private final ClientFixture client;
   private QueryType queryType;
   private String queryText;
@@ -371,24 +390,92 @@ public class QueryBuilder {
   }
 
   /**
+   * Run the query which expect to return vector {@code V} representation
+   * of type {@code T} for the column {@code columnName}.
+   * <p>
+   * <pre>
+   * Example:
+   *
+   *  Set<String> results = queryBuilder()
+   *      .sql(query)
+   *      .vectorValue(
+   *        "columnName",
+   *        SomeVector.class,
+   *        (resultRecordCount, vector) -> {
+   *          Set<String> r = new HashSet<>();
+   *          for (int i = 0; i < resultRecordCount; i++) {
+   *            r.add(vector.getAccessor().getAsStringBuilder(i).toString());
+   *          }
+   *          return r;
+   *        }
+   *      );
+   * </pre>
+   * @param columnName name of the column to read
+   * @param vectorClass returned by the query vector class
+   * @param reader lambda to read the vector value representation
+   * @param <V> vector class
+   * @param <T> return type
+   * @return result produced by {@code reader} lambda or {@code null} if no records returned from the query
+   *
+   */
+  @SuppressWarnings("unchecked")
+  public <T, V> T vectorValue(String columnName, Class<V> vectorClass, VectorQueryReader<T, V> reader)
+      throws RpcException, SchemaChangeException {
+
+    List<QueryDataBatch> result = results();
+    RecordBatchLoader loader = new RecordBatchLoader(client.allocator());
+    QueryDataBatch queryDataBatch = null;
+
+    try {
+      queryDataBatch = result.get(0);
+      loader.load(queryDataBatch.getHeader().getDef(), queryDataBatch.getData());
+
+      V vector = (V) loader.getValueAccessorById(
+          vectorClass,
+          loader.getValueVectorId(SchemaPath.getCompoundPath(columnName)).getFieldIds())
+          .getValueVector();
+
+      return (loader.getRecordCount() > 0) ? reader.read(loader.getRecordCount(), vector) : null;
+    } finally {
+      if (queryDataBatch != null) {
+        queryDataBatch.release();
+      }
+      loader.clear();
+    }
+  }
+
+  /**
    * Run the query that is expected to return (at least) one row
-   * with the only (or first) column returning a long value.
-   * The long value cannot be null.
+   * with the only (or first) column returning a {@link T} value.
+   * The {@link T} value cannot be null.
    *
    * @return the value of the first column of the first row
    * @throws RpcException if anything goes wrong
    */
-
-  public long singletonLong() throws RpcException {
+  private <T> T singletonGeneric(SingletonScalarReader<T> scalarReader) throws RpcException {
     RowSet rowSet = rowSet();
     if (rowSet == null) {
       throw new IllegalStateException("No rows returned");
     }
-    RowSetReader reader = rowSet.reader();
-    reader.next();
-    long value = reader.scalar(0).getLong();
-    rowSet.clear();
-    return value;
+    try {
+      RowSetReader reader = rowSet.reader();
+      reader.next();
+      return scalarReader.read(reader.scalar(0));
+    } finally {
+      rowSet.clear();
+    }
+  }
+
+  /**
+   * Run the query that is expected to return (at least) one row
+   * with the only (or first) column returning a long value.
+   * The long value cannot be null.
+   *
+   * @return the value of the first column of the first row
+   * @throws RpcException if anything goes wrong
+   */
+  public long singletonLong() throws RpcException {
+    return singletonGeneric(ScalarReader::getLong);
   }
 
   /**
@@ -399,17 +486,8 @@ public class QueryBuilder {
    * @return the value of the first column of the first row
    * @throws RpcException if anything goes wrong
    */
-
   public double singletonDouble() throws RpcException {
-    RowSet rowSet = rowSet();
-    if (rowSet == null) {
-      throw new IllegalStateException("No rows returned");
-    }
-    RowSetReader reader = rowSet.reader();
-    reader.next();
-    double value = reader.scalar(0).getDouble();
-    rowSet.clear();
-    return value;
+    return singletonGeneric(ScalarReader::getDouble);
   }
 
   /**
@@ -420,17 +498,20 @@ public class QueryBuilder {
    * @return the value of the first column of the first row
    * @throws RpcException if anything goes wrong
    */
-
   public int singletonInt() throws RpcException {
-    RowSet rowSet = rowSet();
-    if (rowSet == null) {
-      throw new IllegalStateException("No rows returned");
-    }
-    RowSetReader reader = rowSet.reader();
-    reader.next();
-    int value = reader.scalar(0).getInt();
-    rowSet.clear();
-    return value;
+    return singletonGeneric(ScalarReader::getInt);
+  }
+
+  /**
+   * Run the query that is expected to return (at least) one row
+   * with the only (or first) column returning a {@link Period} value.
+   * The {@link Period} value cannot be null.
+   *
+   * @return the value of the first column of the first row
+   * @throws RpcException if anything goes wrong
+   */
+  public Period singletonPeriod() throws RpcException {
+    return singletonGeneric(ScalarReader::getPeriod);
   }
 
   /**
@@ -443,20 +524,7 @@ public class QueryBuilder {
    */
 
   public String singletonString() throws RpcException {
-    RowSet rowSet = rowSet();
-    if (rowSet == null) {
-      throw new IllegalStateException("No rows returned");
-    }
-    RowSetReader reader = rowSet.reader();
-    reader.next();
-    String value;
-    if (reader.scalar(0).isNull()) {
-      value = null;
-    } else {
-      value = reader.scalar(0).getString();
-    }
-    rowSet.clear();
-    return value;
+    return singletonGeneric(ScalarReader::getString);
   }
 
   /**