You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@orc.apache.org by do...@apache.org on 2021/08/16 07:39:39 UTC

[orc] branch main updated: ORC-946: Unified json library (#869)

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

dongjoon pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/orc.git


The following commit(s) were added to refs/heads/main by this push:
     new 2690bd3  ORC-946: Unified json library (#869)
2690bd3 is described below

commit 2690bd3047893d0abf01c65bae830c51ca92e3d7
Author: guiyanakaung <gu...@gmail.com>
AuthorDate: Mon Aug 16 15:39:33 2021 +0800

    ORC-946: Unified json library (#869)
    
    ### What changes were proposed in this pull request?
    
    The java project depends on several json libraries. Replacing jackson/jettison with Gson
    
    ```
    use jackson-core in orc-benchmarks-core
    org.apache.orc.bench.core.convert.json.JsonWriter
    
    use jettison in tools
    org.apache.orc.tools.KeyTool
    org.apache.orc.tools.JsonFileDump
    org.apache.orc.tools.PrintData
    ```
    
    gson vs. jettison has some inconsistent
    
    1.  the scope of character escaping is inconsistent;
        jettison escapes '/'
        gson does not modify
    2. Inconsistent floating point writing;
        jettison removes trailing zeros and decimal points where possible.
        gson leaves it as is.
    3. prettyPrint;
        jettison's prettyPrint is faulty, '[' and '{' are often linked together without a line break.
        gson prettyPrint is fine.
    
    These three behaviours are not forward compatible in this pr.
    
    I don't think prettyPrint needs to be made compatible. The other two I would like to hear your opinion on.
    
    ### Why are the changes needed?
    
    Reduce dependencies and use a unified json lib.
    
    ### How was this patch tested?
    
    Pass the CIs.
---
 java/bench/core/pom.xml                            |    4 -
 .../orc/bench/core/convert/json/JsonWriter.java    |   79 +-
 java/pom.xml                                       |    5 -
 java/tools/pom.xml                                 |    4 -
 .../java/org/apache/orc/tools/JsonFileDump.java    |  294 +++--
 .../src/java/org/apache/orc/tools/KeyTool.java     |   25 +-
 .../src/java/org/apache/orc/tools/PrintData.java   |   60 +-
 .../test/org/apache/orc/tools/TestFileDump.java    |    4 +-
 java/tools/src/test/resources/orc-file-dump.json   | 1184 ++++++++++----------
 9 files changed, 821 insertions(+), 838 deletions(-)

diff --git a/java/bench/core/pom.xml b/java/bench/core/pom.xml
index 762b696..c48a38c 100644
--- a/java/bench/core/pom.xml
+++ b/java/bench/core/pom.xml
@@ -32,10 +32,6 @@
 
   <dependencies>
     <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-core</artifactId>
-    </dependency>
-    <dependency>
       <groupId>com.google.auto.service</groupId>
       <artifactId>auto-service</artifactId>
     </dependency>
diff --git a/java/bench/core/src/java/org/apache/orc/bench/core/convert/json/JsonWriter.java b/java/bench/core/src/java/org/apache/orc/bench/core/convert/json/JsonWriter.java
index 1103984..00b3de2 100644
--- a/java/bench/core/src/java/org/apache/orc/bench/core/convert/json/JsonWriter.java
+++ b/java/bench/core/src/java/org/apache/orc/bench/core/convert/json/JsonWriter.java
@@ -18,8 +18,6 @@
 
 package org.apache.orc.bench.core.convert.json;
 
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonGenerator;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
@@ -47,7 +45,7 @@ import java.util.List;
 
 public class JsonWriter implements BatchWriter {
   private final Writer outStream;
-  private final JsonGenerator writer;
+  private final com.google.gson.stream.JsonWriter writer;
   private final TypeDescription schema;
 
   public JsonWriter(Path path, TypeDescription schema,
@@ -56,45 +54,44 @@ public class JsonWriter implements BatchWriter {
     OutputStream file = path.getFileSystem(conf).create(path, true);
     outStream = new OutputStreamWriter(compression.create(file),
         StandardCharsets.UTF_8);
-    JsonFactory factory = new JsonFactory();
-    factory.setRootValueSeparator("\n");
-    writer = factory.createGenerator(outStream);
+    writer = new com.google.gson.stream.JsonWriter(outStream);
+    writer.setLenient(true);
     this.schema = schema;
   }
 
-  private static void printMap(JsonGenerator writer,
+  private static void printMap(com.google.gson.stream.JsonWriter writer,
                                MapColumnVector vector,
                                TypeDescription schema,
                                int row) throws IOException {
-    writer.writeStartArray();
+    writer.beginArray();
     TypeDescription keyType = schema.getChildren().get(0);
     TypeDescription valueType = schema.getChildren().get(1);
     int offset = (int) vector.offsets[row];
     for (int i = 0; i < vector.lengths[row]; ++i) {
-      writer.writeStartObject();
-      writer.writeFieldName("_key");
+      writer.beginObject();
+      writer.name("_key");
       printValue(writer, vector.keys, keyType, offset + i);
-      writer.writeFieldName("_value");
+      writer.name("_value");
       printValue(writer, vector.values, valueType, offset + i);
-      writer.writeEndObject();
+      writer.endObject();
     }
-    writer.writeEndArray();
+    writer.endArray();
   }
 
-  private static void printList(JsonGenerator writer,
+  private static void printList(com.google.gson.stream.JsonWriter writer,
                                 ListColumnVector vector,
                                 TypeDescription schema,
                                 int row) throws IOException {
-    writer.writeStartArray();
+    writer.beginArray();
     int offset = (int) vector.offsets[row];
     TypeDescription childType = schema.getChildren().get(0);
     for (int i = 0; i < vector.lengths[row]; ++i) {
       printValue(writer, vector.child, childType, offset + i);
     }
-    writer.writeEndArray();
+    writer.endArray();
   }
 
-  private static void printUnion(JsonGenerator writer,
+  private static void printUnion(com.google.gson.stream.JsonWriter writer,
                                  UnionColumnVector vector,
                                  TypeDescription schema,
                                  int row) throws IOException {
@@ -102,21 +99,21 @@ public class JsonWriter implements BatchWriter {
     printValue(writer, vector.fields[tag], schema.getChildren().get(tag), row);
   }
 
-  static void printStruct(JsonGenerator writer,
+  static void printStruct(com.google.gson.stream.JsonWriter writer,
                           StructColumnVector batch,
                           TypeDescription schema,
                           int row) throws IOException {
-    writer.writeStartObject();
+    writer.beginObject();
     List<String> fieldNames = schema.getFieldNames();
     List<TypeDescription> fieldTypes = schema.getChildren();
     for (int i = 0; i < fieldTypes.size(); ++i) {
-      writer.writeFieldName(fieldNames.get(i));
+      writer.name(fieldNames.get(i));
       printValue(writer, batch.fields[i], fieldTypes.get(i), row);
     }
-    writer.writeEndObject();
+    writer.endObject();
   }
 
-  static void printBinary(JsonGenerator writer, BytesColumnVector vector,
+  static void printBinary(com.google.gson.stream.JsonWriter writer, BytesColumnVector vector,
                           int row) throws IOException {
     StringBuilder buffer = new StringBuilder();
     int offset = vector.start[row];
@@ -124,10 +121,10 @@ public class JsonWriter implements BatchWriter {
       int value = 0xff & (int) vector.vector[row][offset + i];
       buffer.append(String.format("%02x", value));
     }
-    writer.writeString(buffer.toString());
+    writer.value(buffer.toString());
   }
 
-  static void printValue(JsonGenerator writer, ColumnVector vector,
+  static void printValue(com.google.gson.stream.JsonWriter writer, ColumnVector vector,
                          TypeDescription schema, int row) throws IOException {
     if (vector.isRepeating) {
       row = 0;
@@ -135,35 +132,35 @@ public class JsonWriter implements BatchWriter {
     if (vector.noNulls || !vector.isNull[row]) {
       switch (schema.getCategory()) {
         case BOOLEAN:
-          writer.writeBoolean(((LongColumnVector) vector).vector[row] != 0);
+          writer.value(((LongColumnVector) vector).vector[row] != 0);
           break;
         case BYTE:
         case SHORT:
         case INT:
         case LONG:
-          writer.writeNumber(((LongColumnVector) vector).vector[row]);
+          writer.value(((LongColumnVector) vector).vector[row]);
           break;
         case FLOAT:
         case DOUBLE:
-          writer.writeNumber(((DoubleColumnVector) vector).vector[row]);
+          writer.value(((DoubleColumnVector) vector).vector[row]);
           break;
         case STRING:
         case CHAR:
         case VARCHAR:
-          writer.writeString(((BytesColumnVector) vector).toString(row));
+          writer.value(((BytesColumnVector) vector).toString(row));
           break;
         case BINARY:
           printBinary(writer, (BytesColumnVector) vector, row);
           break;
         case DECIMAL:
-          writer.writeString(((DecimalColumnVector) vector).vector[row].toString());
+          writer.value(((DecimalColumnVector) vector).vector[row].toString());
           break;
         case DATE:
-          writer.writeString(new DateWritable(
+          writer.value(new DateWritable(
               (int) ((LongColumnVector) vector).vector[row]).toString());
           break;
         case TIMESTAMP:
-          writer.writeString(((TimestampColumnVector) vector)
+          writer.value(((TimestampColumnVector) vector)
               .asScratchTimestamp(row).toString());
           break;
         case LIST:
@@ -179,27 +176,26 @@ public class JsonWriter implements BatchWriter {
           printUnion(writer, (UnionColumnVector) vector, schema, row);
           break;
         default:
-          throw new IllegalArgumentException("Unknown type " +
-              schema.toString());
+          throw new IllegalArgumentException("Unknown type " + schema);
       }
     } else {
-      writer.writeNull();
+      writer.nullValue();
     }
   }
 
-  static void printRow(JsonGenerator writer,
-                              VectorizedRowBatch batch,
-                              TypeDescription schema,
-                              int row) throws IOException {
+  static void printRow(com.google.gson.stream.JsonWriter writer,
+                       VectorizedRowBatch batch,
+                       TypeDescription schema,
+                       int row) throws IOException {
     if (schema.getCategory() == TypeDescription.Category.STRUCT) {
       List<TypeDescription> fieldTypes = schema.getChildren();
       List<String> fieldNames = schema.getFieldNames();
-      writer.writeStartObject();
+      writer.beginObject();
       for (int c = 0; c < batch.cols.length; ++c) {
-        writer.writeFieldName(fieldNames.get(c));
+        writer.name(fieldNames.get(c));
         printValue(writer, batch.cols[c], fieldTypes.get(c), row);
       }
-      writer.writeEndObject();
+      writer.endObject();
     } else {
       printValue(writer, batch.cols[0], schema, row);
     }
@@ -208,6 +204,7 @@ public class JsonWriter implements BatchWriter {
   public void writeBatch(VectorizedRowBatch batch) throws IOException {
     for (int r = 0; r < batch.size; ++r) {
       printRow(writer, batch, schema, r);
+      outStream.write("\n");
     }
   }
 
diff --git a/java/pom.xml b/java/pom.xml
index 64c9647..eafcaf7 100644
--- a/java/pom.xml
+++ b/java/pom.xml
@@ -767,11 +767,6 @@
         </exclusions>
       </dependency>
       <dependency>
-        <groupId>org.codehaus.jettison</groupId>
-        <artifactId>jettison</artifactId>
-        <version>1.4.1</version>
-      </dependency>
-      <dependency>
         <groupId>org.jetbrains</groupId>
         <artifactId>annotations</artifactId>
         <version>21.0.1</version>
diff --git a/java/tools/pom.xml b/java/tools/pom.xml
index 779b739..93584fd 100644
--- a/java/tools/pom.xml
+++ b/java/tools/pom.xml
@@ -75,10 +75,6 @@
       <artifactId>hive-storage-api</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.codehaus.jettison</groupId>
-      <artifactId>jettison</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
     </dependency>
diff --git a/java/tools/src/java/org/apache/orc/tools/JsonFileDump.java b/java/tools/src/java/org/apache/orc/tools/JsonFileDump.java
index 8b6c40a..fba9e20 100644
--- a/java/tools/src/java/org/apache/orc/tools/JsonFileDump.java
+++ b/java/tools/src/java/org/apache/orc/tools/JsonFileDump.java
@@ -17,6 +17,7 @@
  */
 package org.apache.orc.tools;
 
+import com.google.gson.stream.JsonWriter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -45,13 +46,9 @@ import org.apache.orc.impl.ReaderImpl;
 import org.apache.orc.impl.RecordReaderImpl;
 import org.apache.orc.util.BloomFilter;
 import org.apache.orc.util.BloomFilterIO;
-import org.codehaus.jettison.json.JSONArray;
-import org.codehaus.jettison.json.JSONException;
-import org.codehaus.jettison.json.JSONObject;
-import org.codehaus.jettison.json.JSONStringer;
-import org.codehaus.jettison.json.JSONWriter;
 
 import java.io.IOException;
+import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -63,53 +60,57 @@ public class JsonFileDump {
   public static void printJsonMetaData(List<String> files,
       Configuration conf,
       List<Integer> rowIndexCols, boolean prettyPrint, boolean printTimeZone)
-      throws JSONException, IOException {
+      throws IOException {
     if (files.isEmpty()) {
       return;
     }
-    JSONStringer writer = new JSONStringer();
+    StringWriter stringWriter = new StringWriter();
+    JsonWriter writer = new JsonWriter(stringWriter);
+    if (prettyPrint) {
+      writer.setIndent("  ");
+    }
     boolean multiFile = files.size() > 1;
     if (multiFile) {
-      writer.array();
+      writer.beginArray();
     } else {
-      writer.object();
+      writer.beginObject();
     }
     for (String filename : files) {
       try {
         if (multiFile) {
-          writer.object();
+          writer.beginObject();
         }
-        writer.key("fileName").value(filename);
+        writer.name("fileName").value(filename);
         Path path = new Path(filename);
         Reader reader = FileDump.getReader(path, conf, null);
         if (reader == null) {
-          writer.key("status").value("FAILED");
+          writer.name("status").value("FAILED");
           continue;
         }
-        writer.key("fileVersion").value(reader.getFileVersion().getName());
-        writer.key("writerVersion").value(reader.getWriterVersion());
+        writer.name("fileVersion").value(reader.getFileVersion().getName());
+        writer.name("writerVersion").value(reader.getWriterVersion().toString());
         RecordReaderImpl rows = (RecordReaderImpl) reader.rows();
-        writer.key("numberOfRows").value(reader.getNumberOfRows());
-        writer.key("compression").value(reader.getCompressionKind());
+        writer.name("numberOfRows").value(reader.getNumberOfRows());
+        writer.name("compression").value(reader.getCompressionKind().toString());
         if (reader.getCompressionKind() != CompressionKind.NONE) {
-          writer.key("compressionBufferSize").value(reader.getCompressionSize());
+          writer.name("compressionBufferSize").value(reader.getCompressionSize());
         }
-        writer.key("schemaString").value(reader.getSchema().toString());
-        writer.key("schema");
+        writer.name("schemaString").value(reader.getSchema().toString());
+        writer.name("schema");
         writeSchema(writer, reader.getSchema());
-        writer.key("calendar").value(reader.writerUsedProlepticGregorian()
-                                         ? "proleptic Gregorian"
-                                         : "Julian/Gregorian");
-        writer.key("stripeStatistics").array();
+        writer.name("calendar").value(reader.writerUsedProlepticGregorian()
+            ? "proleptic Gregorian"
+            : "Julian/Gregorian");
+        writer.name("stripeStatistics").beginArray();
         List<StripeStatistics> stripeStatistics = reader.getStripeStatistics();
         for (int n = 0; n < stripeStatistics.size(); n++) {
-          writer.object();
-          writer.key("stripeNumber").value(n + 1);
+          writer.beginObject();
+          writer.name("stripeNumber").value(n + 1);
           StripeStatistics ss = stripeStatistics.get(n);
-          writer.key("columnStatistics").array();
+          writer.name("columnStatistics").beginArray();
           for (int i = 0; i < ss.getColumnStatistics().length; i++) {
-            writer.object();
-            writer.key("columnId").value(i);
+            writer.beginObject();
+            writer.name("columnId").value(i);
             writeColumnStatistics(writer, ss.getColumnStatistics()[i]);
             writer.endObject();
           }
@@ -126,53 +127,53 @@ public class JsonFileDump {
             rowIndexCols.add(i);
           }
         }
-        writer.key("fileStatistics").array();
+        writer.name("fileStatistics").beginArray();
         for (int i = 0; i < stats.length; ++i) {
-          writer.object();
-          writer.key("columnId").value(i);
+          writer.beginObject();
+          writer.name("columnId").value(i);
           writeColumnStatistics(writer, stats[i]);
           writer.endObject();
         }
         writer.endArray();
 
-        writer.key("stripes").array();
+        writer.name("stripes").beginArray();
         int stripeIx = -1;
         for (StripeInformation stripe : reader.getStripes()) {
           ++stripeIx;
           long stripeStart = stripe.getOffset();
           OrcProto.StripeFooter footer = rows.readStripeFooter(stripe);
-          writer.object(); // start of stripe information
-          writer.key("stripeNumber").value(stripeIx + 1);
-          writer.key("stripeInformation");
+          writer.beginObject(); // start of stripe information
+          writer.name("stripeNumber").value(stripeIx + 1);
+          writer.name("stripeInformation");
           writeStripeInformation(writer, stripe);
           if (printTimeZone) {
-            writer.key("writerTimezone").value(
+            writer.name("writerTimezone").value(
                 footer.hasWriterTimezone() ? footer.getWriterTimezone() : FileDump.UNKNOWN);
           }
           long sectionStart = stripeStart;
 
-          writer.key("streams").array();
+          writer.name("streams").beginArray();
           for (OrcProto.Stream section : footer.getStreamsList()) {
-            writer.object();
+            writer.beginObject();
             String kind = section.hasKind() ? section.getKind().name() : FileDump.UNKNOWN;
-            writer.key("columnId").value(section.getColumn());
-            writer.key("section").value(kind);
-            writer.key("startOffset").value(sectionStart);
-            writer.key("length").value(section.getLength());
+            writer.name("columnId").value(section.getColumn());
+            writer.name("section").value(kind);
+            writer.name("startOffset").value(sectionStart);
+            writer.name("length").value(section.getLength());
             sectionStart += section.getLength();
             writer.endObject();
           }
           writer.endArray();
 
-          writer.key("encodings").array();
+          writer.name("encodings").beginArray();
           for (int i = 0; i < footer.getColumnsCount(); ++i) {
-            writer.object();
+            writer.beginObject();
             OrcProto.ColumnEncoding encoding = footer.getColumns(i);
-            writer.key("columnId").value(i);
-            writer.key("kind").value(encoding.getKind());
+            writer.name("columnId").value(i);
+            writer.name("kind").value(encoding.getKind().toString());
             if (encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY ||
                 encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY_V2) {
-              writer.key("dictionarySize").value(encoding.getDictionarySize());
+              writer.name("dictionarySize").value(encoding.getDictionarySize());
             }
             writer.endObject();
           }
@@ -185,10 +186,10 @@ public class JsonFileDump {
               sargColumns[colIdx] = true;
             }
             OrcIndex indices = rows.readRowIndex(stripeIx, null, sargColumns);
-            writer.key("indexes").array();
+            writer.name("indexes").beginArray();
             for (int col : rowIndexCols) {
-              writer.object();
-              writer.key("columnId").value(col);
+              writer.beginObject();
+              writer.name("columnId").value(col);
               writeRowGroupIndexes(writer, col, indices.getRowGroupIndex(),
                   reader.getSchema(), (ReaderImpl) reader);
               writeBloomFilterIndexes(writer, col, indices,
@@ -208,94 +209,81 @@ public class JsonFileDump {
         long paddedBytes = FileDump.getTotalPaddingSize(reader);
         // empty ORC file is ~45 bytes. Assumption here is file length always >0
         double percentPadding = ((double) paddedBytes / (double) fileLen) * 100;
-        writer.key("fileLength").value(fileLen);
-        writer.key("paddingLength").value(paddedBytes);
-        writer.key("paddingRatio").value(percentPadding);
+        writer.name("fileLength").value(fileLen);
+        writer.name("paddingLength").value(paddedBytes);
+        writer.name("paddingRatio").value(percentPadding);
         AcidStats acidStats = OrcAcidUtils.parseAcidStats(reader);
         if (acidStats != null) {
-          writer.key("numInserts").value(acidStats.inserts);
-          writer.key("numDeletes").value(acidStats.deletes);
-          writer.key("numUpdates").value(acidStats.updates);
+          writer.name("numInserts").value(acidStats.inserts);
+          writer.name("numDeletes").value(acidStats.deletes);
+          writer.name("numUpdates").value(acidStats.updates);
         }
-        writer.key("status").value("OK");
+        writer.name("status").value("OK");
         rows.close();
 
         writer.endObject();
       } catch (Throwable e) {
-        writer.key("status").value("FAILED");
+        writer.name("status").value("FAILED");
         throw e;
       }
     }
     if (multiFile) {
       writer.endArray();
     }
-
-    if (prettyPrint) {
-      final String prettyJson;
-      if (multiFile) {
-        JSONArray jsonArray = new JSONArray(writer.toString());
-        prettyJson = jsonArray.toString(2);
-      } else {
-        JSONObject jsonObject = new JSONObject(writer.toString());
-        prettyJson = jsonObject.toString(2);
-      }
-      System.out.println(prettyJson);
-    } else {
-      System.out.println(writer.toString());
-    }
+    System.out.println(stringWriter);
   }
 
-  private static void writeSchema(JSONStringer writer, TypeDescription type)
-      throws JSONException {
-    writer.object();
-    writer.key("columnId").value(type.getId());
-    writer.key("columnType").value(type.getCategory());
+  private static void writeSchema(JsonWriter writer, TypeDescription type)
+      throws IOException {
+    writer.beginObject();
+    writer.name("columnId").value(type.getId());
+    writer.name("columnType").value(type.getCategory().toString());
     List<String> attributes = type.getAttributeNames();
     if (attributes.size() > 0) {
-      writer.key("attributes").object();
+      writer.name("attributes").beginObject();
       for (String name : attributes) {
-        writer.key(name).value(type.getAttributeValue(name));
+        writer.name(name).value(type.getAttributeValue(name));
       }
       writer.endObject();
     }
     switch (type.getCategory()) {
       case DECIMAL:
-        writer.key("precision").value(type.getPrecision());
-        writer.key("scale").value(type.getScale());
+        writer.name("precision").value(type.getPrecision());
+        writer.name("scale").value(type.getScale());
         break;
       case VARCHAR:
       case CHAR:
-        writer.key("maxLength").value(type.getMaxLength());
+        writer.name("maxLength").value(type.getMaxLength());
         break;
       default:
         break;
     }
     List<TypeDescription> children = type.getChildren();
     if (children != null) {
-      writer.key("children");
+      writer.name("children");
       switch (type.getCategory()) {
         case STRUCT:
-          writer.object();
+          writer.beginObject();
           List<String> fields = type.getFieldNames();
           for (int c = 0; c < fields.size(); ++c) {
-            writer.key(fields.get(c));
+            writer.name(fields.get(c));
             writeSchema(writer, children.get(c));
           }
           writer.endObject();
           break;
         case LIST:
-          writer.array();
+          writer.beginArray();
           writeSchema(writer, children.get(0));
           writer.endArray();
           break;
         case MAP:
-          writer.array();
+          writer.beginArray();
           writeSchema(writer, children.get(0));
           writeSchema(writer, children.get(1));
           writer.endArray();
           break;
         case UNION:
-          writer.array();
+          writer.beginArray();
           for (TypeDescription child : children) {
             writeSchema(writer, child);
           }
@@ -308,101 +296,101 @@ public class JsonFileDump {
     writer.endObject();
   }
 
-  private static void writeStripeInformation(JSONWriter writer, StripeInformation stripe)
-      throws JSONException {
-    writer.object();
-    writer.key("offset").value(stripe.getOffset());
-    writer.key("indexLength").value(stripe.getIndexLength());
-    writer.key("dataLength").value(stripe.getDataLength());
-    writer.key("footerLength").value(stripe.getFooterLength());
-    writer.key("rowCount").value(stripe.getNumberOfRows());
+  private static void writeStripeInformation(JsonWriter writer, StripeInformation stripe)
+      throws IOException {
+    writer.beginObject();
+    writer.name("offset").value(stripe.getOffset());
+    writer.name("indexLength").value(stripe.getIndexLength());
+    writer.name("dataLength").value(stripe.getDataLength());
+    writer.name("footerLength").value(stripe.getFooterLength());
+    writer.name("rowCount").value(stripe.getNumberOfRows());
     writer.endObject();
   }
 
-  private static void writeColumnStatistics(JSONWriter writer, ColumnStatistics cs)
-      throws JSONException {
+  private static void writeColumnStatistics(JsonWriter writer, ColumnStatistics cs)
+      throws IOException {
     if (cs != null) {
-      writer.key("count").value(cs.getNumberOfValues());
-      writer.key("hasNull").value(cs.hasNull());
+      writer.name("count").value(cs.getNumberOfValues());
+      writer.name("hasNull").value(cs.hasNull());
       if (cs.getBytesOnDisk() != 0) {
-        writer.key("bytesOnDisk").value(cs.getBytesOnDisk());
+        writer.name("bytesOnDisk").value(cs.getBytesOnDisk());
       }
       if (cs instanceof BinaryColumnStatistics) {
-        writer.key("totalLength").value(((BinaryColumnStatistics) cs).getSum());
-        writer.key("type").value(OrcProto.Type.Kind.BINARY);
+        writer.name("totalLength").value(((BinaryColumnStatistics) cs).getSum());
+        writer.name("type").value(OrcProto.Type.Kind.BINARY.toString());
       } else if (cs instanceof BooleanColumnStatistics) {
-        writer.key("trueCount").value(((BooleanColumnStatistics) cs).getTrueCount());
-        writer.key("falseCount").value(((BooleanColumnStatistics) cs).getFalseCount());
-        writer.key("type").value(OrcProto.Type.Kind.BOOLEAN);
+        writer.name("trueCount").value(((BooleanColumnStatistics) cs).getTrueCount());
+        writer.name("falseCount").value(((BooleanColumnStatistics) cs).getFalseCount());
+        writer.name("type").value(OrcProto.Type.Kind.BOOLEAN.toString());
       } else if (cs instanceof IntegerColumnStatistics) {
-        writer.key("min").value(((IntegerColumnStatistics) cs).getMinimum());
-        writer.key("max").value(((IntegerColumnStatistics) cs).getMaximum());
+        writer.name("min").value(((IntegerColumnStatistics) cs).getMinimum());
+        writer.name("max").value(((IntegerColumnStatistics) cs).getMaximum());
         if (((IntegerColumnStatistics) cs).isSumDefined()) {
-          writer.key("sum").value(((IntegerColumnStatistics) cs).getSum());
+          writer.name("sum").value(((IntegerColumnStatistics) cs).getSum());
         }
-        writer.key("type").value(OrcProto.Type.Kind.LONG);
+        writer.name("type").value(OrcProto.Type.Kind.LONG.toString());
       } else if (cs instanceof DoubleColumnStatistics) {
-        writer.key("min").value(((DoubleColumnStatistics) cs).getMinimum());
-        writer.key("max").value(((DoubleColumnStatistics) cs).getMaximum());
-        writer.key("sum").value(((DoubleColumnStatistics) cs).getSum());
-        writer.key("type").value(OrcProto.Type.Kind.DOUBLE);
+        writer.name("min").value(((DoubleColumnStatistics) cs).getMinimum());
+        writer.name("max").value(((DoubleColumnStatistics) cs).getMaximum());
+        writer.name("sum").value(((DoubleColumnStatistics) cs).getSum());
+        writer.name("type").value(OrcProto.Type.Kind.DOUBLE.toString());
       } else if (cs instanceof StringColumnStatistics) {
         String lower = ((StringColumnStatistics) cs).getLowerBound();
         if (((StringColumnStatistics) cs).getMinimum() != null) {
-          writer.key("min").value(lower);
+          writer.name("min").value(lower);
         } else if (lower != null) {
-          writer.key("lowerBound").value(lower);
+          writer.name("lowerBound").value(lower);
         }
         String upper = ((StringColumnStatistics) cs).getUpperBound();
         if (((StringColumnStatistics) cs).getMaximum() != null) {
-          writer.key("max").value(upper);
+          writer.name("max").value(upper);
         } else if (upper != null) {
-          writer.key("upperBound").value(upper);
+          writer.name("upperBound").value(upper);
         }
-        writer.key("totalLength").value(((StringColumnStatistics) cs).getSum());
-        writer.key("type").value(OrcProto.Type.Kind.STRING);
+        writer.name("totalLength").value(((StringColumnStatistics) cs).getSum());
+        writer.name("type").value(OrcProto.Type.Kind.STRING.toString());
       } else if (cs instanceof DateColumnStatistics) {
         if (((DateColumnStatistics) cs).getMaximumLocalDate() != null) {
-          writer.key("min").value(((DateColumnStatistics) cs).getMinimumLocalDate());
-          writer.key("max").value(((DateColumnStatistics) cs).getMaximumLocalDate());
+          writer.name("min").value(((DateColumnStatistics) cs).getMinimumLocalDate().toString());
+          writer.name("max").value(((DateColumnStatistics) cs).getMaximumLocalDate().toString());
         }
-        writer.key("type").value(OrcProto.Type.Kind.DATE);
+        writer.name("type").value(OrcProto.Type.Kind.DATE.toString());
       } else if (cs instanceof TimestampColumnStatistics) {
         if (((TimestampColumnStatistics) cs).getMaximum() != null) {
-          writer.key("min").value(((TimestampColumnStatistics) cs).getMinimum());
-          writer.key("max").value(((TimestampColumnStatistics) cs).getMaximum());
+          writer.name("min").value(((TimestampColumnStatistics) cs).getMinimum().toString());
+          writer.name("max").value(((TimestampColumnStatistics) cs).getMaximum().toString());
         }
-        writer.key("type").value(OrcProto.Type.Kind.TIMESTAMP);
+        writer.name("type").value(OrcProto.Type.Kind.TIMESTAMP.toString());
       } else if (cs instanceof DecimalColumnStatistics) {
         if (((DecimalColumnStatistics) cs).getMaximum() != null) {
-          writer.key("min").value(((DecimalColumnStatistics) cs).getMinimum());
-          writer.key("max").value(((DecimalColumnStatistics) cs).getMaximum());
-          writer.key("sum").value(((DecimalColumnStatistics) cs).getSum());
+          writer.name("min").value(((DecimalColumnStatistics) cs).getMinimum().toString());
+          writer.name("max").value(((DecimalColumnStatistics) cs).getMaximum().toString());
+          writer.name("sum").value(((DecimalColumnStatistics) cs).getSum().toString());
         }
-        writer.key("type").value(OrcProto.Type.Kind.DECIMAL);
+        writer.name("type").value(OrcProto.Type.Kind.DECIMAL.toString());
       } else if (cs instanceof CollectionColumnStatistics) {
-        writer.key("minChildren").value(((CollectionColumnStatistics) cs).getMinimumChildren());
-        writer.key("maxChildren").value(((CollectionColumnStatistics) cs).getMaximumChildren());
-        writer.key("totalChildren").value(((CollectionColumnStatistics) cs).getTotalChildren());
+        writer.name("minChildren").value(((CollectionColumnStatistics) cs).getMinimumChildren());
+        writer.name("maxChildren").value(((CollectionColumnStatistics) cs).getMaximumChildren());
+        writer.name("totalChildren").value(((CollectionColumnStatistics) cs).getTotalChildren());
       }
     }
   }
 
-  private static void writeBloomFilterIndexes(JSONWriter writer, int col,
+  private static void writeBloomFilterIndexes(JsonWriter writer, int col,
                                               OrcIndex index,
                                               OrcFile.WriterVersion version,
                                               TypeDescription.Category type,
                                               OrcProto.ColumnEncoding encoding
-                                              ) throws JSONException {
+  ) throws IOException {
 
     BloomFilter stripeLevelBF = null;
     OrcProto.BloomFilterIndex[] bloomFilterIndex = index.getBloomFilterIndex();
     if (bloomFilterIndex != null && bloomFilterIndex[col] != null) {
       int entryIx = 0;
-      writer.key("bloomFilterIndexes").array();
+      writer.name("bloomFilterIndexes").beginArray();
       for (OrcProto.BloomFilter bf : bloomFilterIndex[col].getBloomFilterList()) {
-        writer.object();
-        writer.key("entryId").value(entryIx++);
+        writer.beginObject();
+        writer.name("entryId").value(entryIx++);
         BloomFilter toMerge = BloomFilterIO.deserialize(
             index.getBloomFilterKinds()[col], encoding, version, type, bf);
         writeBloomFilterStats(writer, toMerge);
@@ -416,15 +404,15 @@ public class JsonFileDump {
       writer.endArray();
     }
     if (stripeLevelBF != null) {
-      writer.key("stripeLevelBloomFilter");
-      writer.object();
+      writer.name("stripeLevelBloomFilter");
+      writer.beginObject();
       writeBloomFilterStats(writer, stripeLevelBF);
       writer.endObject();
     }
   }
 
-  private static void writeBloomFilterStats(JSONWriter writer, BloomFilter bf)
-      throws JSONException {
+  private static void writeBloomFilterStats(JsonWriter writer, BloomFilter bf)
+      throws IOException {
     int bitCount = bf.getBitSize();
     int popCount = 0;
     for (long l : bf.getBitSet()) {
@@ -433,27 +421,27 @@ public class JsonFileDump {
     int k = bf.getNumHashFunctions();
     float loadFactor = (float) popCount / (float) bitCount;
     float expectedFpp = (float) Math.pow(loadFactor, k);
-    writer.key("numHashFunctions").value(k);
-    writer.key("bitCount").value(bitCount);
-    writer.key("popCount").value(popCount);
-    writer.key("loadFactor").value(loadFactor);
-    writer.key("expectedFpp").value(expectedFpp);
+    writer.name("numHashFunctions").value(k);
+    writer.name("bitCount").value(bitCount);
+    writer.name("popCount").value(popCount);
+    writer.name("loadFactor").value(loadFactor);
+    writer.name("expectedFpp").value(expectedFpp);
   }
 
-  private static void writeRowGroupIndexes(JSONWriter writer, int col,
+  private static void writeRowGroupIndexes(JsonWriter writer, int col,
                                            OrcProto.RowIndex[] rowGroupIndex,
                                            TypeDescription schema,
-                                           ReaderImpl reader) throws JSONException {
+                                           ReaderImpl reader) throws IOException {
     OrcProto.RowIndex index;
     if (rowGroupIndex == null || (col >= rowGroupIndex.length) ||
         ((index = rowGroupIndex[col]) == null)) {
       return;
     }
 
-    writer.key("rowGroupIndexes").array();
+    writer.name("rowGroupIndexes").beginArray();
     for (int entryIx = 0; entryIx < index.getEntryCount(); ++entryIx) {
-      writer.object();
-      writer.key("entryId").value(entryIx);
+      writer.beginObject();
+      writer.name("entryId").value(entryIx);
       OrcProto.RowIndexEntry entry = index.getEntry(entryIx);
       if (entry == null) {
         continue;
@@ -462,7 +450,7 @@ public class JsonFileDump {
       writeColumnStatistics(writer, ColumnStatisticsImpl.deserialize(
           schema.findSubtype(col), colStats, reader.writerUsedProlepticGregorian(),
           reader.getConvertToProlepticGregorian()));
-      writer.key("positions").array();
+      writer.name("positions").beginArray();
       for (int posIx = 0; posIx < entry.getPositionsCount(); ++posIx) {
         writer.value(entry.getPositions(posIx));
       }
diff --git a/java/tools/src/java/org/apache/orc/tools/KeyTool.java b/java/tools/src/java/org/apache/orc/tools/KeyTool.java
index ea592bc..03c649e 100644
--- a/java/tools/src/java/org/apache/orc/tools/KeyTool.java
+++ b/java/tools/src/java/org/apache/orc/tools/KeyTool.java
@@ -17,6 +17,7 @@
  */
 package org.apache.orc.tools;
 
+import com.google.gson.stream.JsonWriter;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.DefaultParser;
 import org.apache.commons.cli.HelpFormatter;
@@ -29,8 +30,6 @@ import org.apache.orc.EncryptionAlgorithm;
 import org.apache.orc.impl.CryptoUtils;
 import org.apache.orc.impl.HadoopShims;
 import org.apache.orc.impl.KeyProvider;
-import org.codehaus.jettison.json.JSONException;
-import org.codehaus.jettison.json.JSONWriter;
 
 import java.io.IOException;
 import java.io.OutputStreamWriter;
@@ -42,23 +41,23 @@ import java.security.SecureRandom;
  */
 public class KeyTool {
 
-  static void printKey(JSONWriter writer,
+  static void printKey(JsonWriter writer,
                        KeyProvider provider,
-                       String keyName) throws JSONException, IOException {
+                       String keyName) throws IOException {
     HadoopShims.KeyMetadata meta = provider.getCurrentKeyVersion(keyName);
-    writer.object();
-    writer.key("name");
+    writer.beginObject();
+    writer.name("name");
     writer.value(keyName);
     EncryptionAlgorithm algorithm = meta.getAlgorithm();
-    writer.key("algorithm");
+    writer.name("algorithm");
     writer.value(algorithm.getAlgorithm());
-    writer.key("keyLength");
+    writer.name("keyLength");
     writer.value(algorithm.keyLength());
-    writer.key("version");
+    writer.name("version");
     writer.value(meta.getVersion());
     byte[] iv = new byte[algorithm.getIvLength()];
     byte[] key = provider.decryptLocalKey(meta, iv).getEncoded();
-    writer.key("key 0");
+    writer.name("key 0");
     writer.value(new BytesWritable(key).toString());
     writer.endObject();
   }
@@ -79,7 +78,7 @@ public class KeyTool {
     this.conf = conf;
   }
 
-  void run() throws IOException, JSONException {
+  void run() throws IOException {
     KeyProvider provider =
         CryptoUtils.getKeyProvider(conf, new SecureRandom());
     if (provider == null) {
@@ -87,7 +86,7 @@ public class KeyTool {
       System.exit(1);
     }
     for(String keyName: provider.getKeyNames()) {
-      JSONWriter writer = new JSONWriter(this.writer);
+      JsonWriter writer = new JsonWriter(this.writer);
       printKey(writer, provider, keyName);
       this.writer.write('\n');
     }
@@ -113,7 +112,7 @@ public class KeyTool {
 
   public static void main(Configuration conf,
                           String[] args
-                          ) throws IOException, ParseException, JSONException {
+                          ) throws IOException, ParseException {
     KeyTool tool = new KeyTool(conf, args);
     tool.run();
   }
diff --git a/java/tools/src/java/org/apache/orc/tools/PrintData.java b/java/tools/src/java/org/apache/orc/tools/PrintData.java
index 875a6a2..dc33b7d 100644
--- a/java/tools/src/java/org/apache/orc/tools/PrintData.java
+++ b/java/tools/src/java/org/apache/orc/tools/PrintData.java
@@ -17,6 +17,7 @@
  */
 package org.apache.orc.tools;
 
+import com.google.gson.stream.JsonWriter;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.DefaultParser;
 import org.apache.commons.cli.HelpFormatter;
@@ -40,8 +41,6 @@ import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.orc.Reader;
 import org.apache.orc.RecordReader;
 import org.apache.orc.TypeDescription;
-import org.codehaus.jettison.json.JSONException;
-import org.codehaus.jettison.json.JSONWriter;
 
 import java.io.IOException;
 import java.io.OutputStreamWriter;
@@ -57,30 +56,30 @@ import java.util.Optional;
  */
 public class PrintData {
 
-  private static void printMap(JSONWriter writer,
+  private static void printMap(JsonWriter writer,
                                MapColumnVector vector,
                                TypeDescription schema,
-                               int row) throws JSONException {
-    writer.array();
+                               int row) throws IOException {
+    writer.beginArray();
     TypeDescription keyType = schema.getChildren().get(0);
     TypeDescription valueType = schema.getChildren().get(1);
     int offset = (int) vector.offsets[row];
     for (int i = 0; i < vector.lengths[row]; ++i) {
-      writer.object();
-      writer.key("_key");
+      writer.beginObject();
+      writer.name("_key");
       printValue(writer, vector.keys, keyType, offset + i);
-      writer.key("_value");
+      writer.name("_value");
       printValue(writer, vector.values, valueType, offset + i);
       writer.endObject();
     }
     writer.endArray();
   }
 
-  private static void printList(JSONWriter writer,
+  private static void printList(JsonWriter writer,
                                 ListColumnVector vector,
                                 TypeDescription schema,
-                                int row) throws JSONException {
-    writer.array();
+                                int row) throws IOException {
+    writer.beginArray();
     int offset = (int) vector.offsets[row];
     TypeDescription childType = schema.getChildren().get(0);
     for (int i = 0; i < vector.lengths[row]; ++i) {
@@ -89,39 +88,39 @@ public class PrintData {
     writer.endArray();
   }
 
-  private static void printUnion(JSONWriter writer,
+  private static void printUnion(JsonWriter writer,
                                  UnionColumnVector vector,
                                  TypeDescription schema,
-                                 int row) throws JSONException {
+                                 int row) throws IOException {
     int tag = vector.tags[row];
     printValue(writer, vector.fields[tag], schema.getChildren().get(tag), row);
   }
 
-  static void printStruct(JSONWriter writer,
+  static void printStruct(JsonWriter writer,
                           StructColumnVector batch,
                           TypeDescription schema,
-                          int row) throws JSONException {
-    writer.object();
+                          int row) throws IOException {
+    writer.beginObject();
     List<String> fieldNames = schema.getFieldNames();
     List<TypeDescription> fieldTypes = schema.getChildren();
     for (int i = 0; i < fieldTypes.size(); ++i) {
-      writer.key(fieldNames.get(i));
+      writer.name(fieldNames.get(i));
       printValue(writer, batch.fields[i], fieldTypes.get(i), row);
     }
     writer.endObject();
   }
 
-  static void printBinary(JSONWriter writer, BytesColumnVector vector,
-                          int row) throws JSONException {
-    writer.array();
+  static void printBinary(JsonWriter writer, BytesColumnVector vector,
+                          int row) throws IOException {
+    writer.beginArray();
     int offset = vector.start[row];
     for(int i=0; i < vector.length[row]; ++i) {
       writer.value(0xff & (int) vector.vector[row][offset + i]);
     }
     writer.endArray();
   }
-  static void printValue(JSONWriter writer, ColumnVector vector,
-                         TypeDescription schema, int row) throws JSONException {
+  static void printValue(JsonWriter writer, ColumnVector vector,
+                         TypeDescription schema, int row) throws IOException {
     if (vector.isRepeating) {
       row = 0;
     }
@@ -177,20 +176,20 @@ public class PrintData {
               schema.toString());
       }
     } else {
-      writer.value(null);
+      writer.nullValue();
     }
   }
 
-  static void printRow(JSONWriter writer,
+  static void printRow(JsonWriter writer,
                        VectorizedRowBatch batch,
                        TypeDescription schema,
-                       int row) throws JSONException {
+                       int row) throws IOException {
     if (schema.getCategory() == TypeDescription.Category.STRUCT) {
       List<TypeDescription> fieldTypes = schema.getChildren();
       List<String> fieldNames = schema.getFieldNames();
-      writer.object();
+      writer.beginObject();
       for (int c = 0; c < batch.cols.length; ++c) {
-        writer.key(fieldNames.get(c));
+        writer.name(fieldNames.get(c));
         printValue(writer, batch.cols[c], fieldTypes.get(c), row);
       }
       writer.endObject();
@@ -200,7 +199,7 @@ public class PrintData {
   }
 
   static void printJsonData(PrintStream printStream,
-          Reader reader, Optional<Integer> numberOfRows) throws IOException, JSONException {
+          Reader reader, Optional<Integer> numberOfRows) throws IOException {
     OutputStreamWriter out = new OutputStreamWriter(printStream, StandardCharsets.UTF_8);
     RecordReader rows = reader.rows();
     try {
@@ -212,7 +211,7 @@ public class PrintData {
             break;
         }
         for (int r=0; r < batch.size; ++r) {
-          JSONWriter writer = new JSONWriter(out);
+          JsonWriter writer = new JsonWriter(out);
           printRow(writer, batch, schema, r);
           out.write("\n");
           out.flush();
@@ -267,7 +266,7 @@ public class PrintData {
   }
 
   static void main(Configuration conf, String[] args
-                   ) throws IOException, JSONException, ParseException {
+                   ) throws ParseException {
     CommandLine cli = parseCommandLine(args);
     if (cli.hasOption('h') || cli.getArgs().length == 0) {
       printHelp();
@@ -291,7 +290,6 @@ public class PrintData {
         } catch (Exception e) {
           System.err.println("Unable to dump data for file: " + file);
           e.printStackTrace();
-          continue;
         }
       }
     }
diff --git a/java/tools/src/test/org/apache/orc/tools/TestFileDump.java b/java/tools/src/test/org/apache/orc/tools/TestFileDump.java
index e8c46a1..3cbdfe7 100644
--- a/java/tools/src/test/org/apache/orc/tools/TestFileDump.java
+++ b/java/tools/src/test/org/apache/orc/tools/TestFileDump.java
@@ -333,8 +333,8 @@ public class TestFileDump {
     System.out.flush();
     System.setOut(origOut);
     String[] lines = myOut.toString(StandardCharsets.UTF_8.toString()).split("\n");
-    assertEquals("{\"b\":true,\"bt\":10,\"s\":100,\"i\":1000,\"l\":10000,\"f\":4,\"d\":20,\"de\":\"4.2222\",\"t\":\"2014-11-25 18:09:24.0\",\"dt\":\"2014-11-25\",\"str\":\"string\",\"c\":\"hello\",\"vc\":\"hello\",\"m\":[{\"_key\":\"k1\",\"_value\":\"v1\"}],\"a\":[100,200],\"st\":{\"i\":10,\"s\":\"foo\"}}", lines[0]);
-    assertEquals("{\"b\":false,\"bt\":20,\"s\":200,\"i\":2000,\"l\":20000,\"f\":8,\"d\":40,\"de\":\"2.2222\",\"t\":\"2014-11-25 18:02:44.0\",\"dt\":\"2014-09-28\",\"str\":\"abcd\",\"c\":\"world\",\"vc\":\"world\",\"m\":[{\"_key\":\"k3\",\"_value\":\"v3\"}],\"a\":[200,300],\"st\":{\"i\":20,\"s\":\"bar\"}}", lines[1]);
+    assertEquals("{\"b\":true,\"bt\":10,\"s\":100,\"i\":1000,\"l\":10000,\"f\":4.0,\"d\":20.0,\"de\":\"4.2222\",\"t\":\"2014-11-25 18:09:24.0\",\"dt\":\"2014-11-25\",\"str\":\"string\",\"c\":\"hello\",\"vc\":\"hello\",\"m\":[{\"_key\":\"k1\",\"_value\":\"v1\"}],\"a\":[100,200],\"st\":{\"i\":10,\"s\":\"foo\"}}", lines[0]);
+    assertEquals("{\"b\":false,\"bt\":20,\"s\":200,\"i\":2000,\"l\":20000,\"f\":8.0,\"d\":40.0,\"de\":\"2.2222\",\"t\":\"2014-11-25 18:02:44.0\",\"dt\":\"2014-09-28\",\"str\":\"abcd\",\"c\":\"world\",\"vc\":\"world\",\"m\":[{\"_key\":\"k3\",\"_value\":\"v3\"}],\"a\":[200,300],\"st\":{\"i\":20,\"s\":\"bar\"}}", lines[1]);
   }
 
   // Test that if the fraction of rows that have distinct strings is greater than the configured
diff --git a/java/tools/src/test/resources/orc-file-dump.json b/java/tools/src/test/resources/orc-file-dump.json
index 3545efe..85cfa2a 100644
--- a/java/tools/src/test/resources/orc-file-dump.json
+++ b/java/tools/src/test/resources/orc-file-dump.json
@@ -28,7 +28,7 @@
       }
     }
   },
-  "calendar": "Julian\/Gregorian",
+  "calendar": "Julian/Gregorian",
   "stripeStatistics": [
     {
       "stripeNumber": 1,
@@ -359,150 +359,152 @@
           "dictionarySize": 35
         }
       ],
-      "indexes": [{
-        "columnId": 3,
-        "rowGroupIndexes": [
-          {
-            "entryId": 0,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3873,
-            "type": "STRING",
-            "positions": [
-              0,
-              0,
-              0,
-              0,
-              0,
-              0,
-              0
-            ]
-          },
-          {
-            "entryId": 1,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3861,
-            "type": "STRING",
-            "positions": [
-              0,
-              38,
-              12,
-              0,
-              0,
-              736,
-              23
-            ]
-          },
-          {
-            "entryId": 2,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3946,
-            "type": "STRING",
-            "positions": [
-              0,
-              78,
-              12,
-              0,
-              0,
-              1473,
-              43
-            ]
-          },
-          {
-            "entryId": 3,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3774,
-            "type": "STRING",
-            "positions": [
-              0,
-              118,
-              12,
-              0,
-              0,
-              2067,
-              261
-            ]
-          },
-          {
-            "entryId": 4,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3829,
-            "type": "STRING",
-            "positions": [
-              0,
-              158,
-              12,
-              0,
-              0,
-              2992,
-              35
-            ]
-          }
-        ],
-        "bloomFilterIndexes": [
-          {
-            "entryId": 0,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 1,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 2,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 3,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 4,
+      "indexes": [
+        {
+          "columnId": 3,
+          "rowGroupIndexes": [
+            {
+              "entryId": 0,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3873,
+              "type": "STRING",
+              "positions": [
+                0,
+                0,
+                0,
+                0,
+                0,
+                0,
+                0
+              ]
+            },
+            {
+              "entryId": 1,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3861,
+              "type": "STRING",
+              "positions": [
+                0,
+                38,
+                12,
+                0,
+                0,
+                736,
+                23
+              ]
+            },
+            {
+              "entryId": 2,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3946,
+              "type": "STRING",
+              "positions": [
+                0,
+                78,
+                12,
+                0,
+                0,
+                1473,
+                43
+              ]
+            },
+            {
+              "entryId": 3,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3774,
+              "type": "STRING",
+              "positions": [
+                0,
+                118,
+                12,
+                0,
+                0,
+                2067,
+                261
+              ]
+            },
+            {
+              "entryId": 4,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3829,
+              "type": "STRING",
+              "positions": [
+                0,
+                158,
+                12,
+                0,
+                0,
+                2992,
+                35
+              ]
+            }
+          ],
+          "bloomFilterIndexes": [
+            {
+              "entryId": 0,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            },
+            {
+              "entryId": 1,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            },
+            {
+              "entryId": 2,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            },
+            {
+              "entryId": 3,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            },
+            {
+              "entryId": 4,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            }
+          ],
+          "stripeLevelBloomFilter": {
             "numHashFunctions": 4,
             "bitCount": 6272,
             "popCount": 138,
             "loadFactor": 0.022002551704645157,
             "expectedFpp": 2.3436470542037569E-7
           }
-        ],
-        "stripeLevelBloomFilter": {
-          "numHashFunctions": 4,
-          "bitCount": 6272,
-          "popCount": 138,
-          "loadFactor": 0.022002551704645157,
-          "expectedFpp": 2.3436470542037569E-7
         }
-      }]
+      ]
     },
     {
       "stripeNumber": 2,
@@ -600,150 +602,152 @@
           "dictionarySize": 35
         }
       ],
-      "indexes": [{
-        "columnId": 3,
-        "rowGroupIndexes": [
-          {
-            "entryId": 0,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3946,
-            "type": "STRING",
-            "positions": [
-              0,
-              0,
-              0,
-              0,
-              0,
-              0,
-              0
-            ]
-          },
-          {
-            "entryId": 1,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3836,
-            "type": "STRING",
-            "positions": [
-              0,
-              38,
-              12,
-              0,
-              0,
-              746,
-              11
-            ]
-          },
-          {
-            "entryId": 2,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3791,
-            "type": "STRING",
-            "positions": [
-              0,
-              78,
-              12,
-              0,
-              0,
-              1430,
-              95
-            ]
-          },
-          {
-            "entryId": 3,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3904,
-            "type": "STRING",
-            "positions": [
-              0,
-              118,
-              12,
-              0,
-              0,
-              2239,
-              23
-            ]
-          },
-          {
-            "entryId": 4,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3920,
-            "type": "STRING",
-            "positions": [
-              0,
-              158,
-              12,
-              0,
-              0,
-              2994,
-              17
-            ]
-          }
-        ],
-        "bloomFilterIndexes": [
-          {
-            "entryId": 0,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 1,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 2,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 3,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 4,
+      "indexes": [
+        {
+          "columnId": 3,
+          "rowGroupIndexes": [
+            {
+              "entryId": 0,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3946,
+              "type": "STRING",
+              "positions": [
+                0,
+                0,
+                0,
+                0,
+                0,
+                0,
+                0
+              ]
+            },
+            {
+              "entryId": 1,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3836,
+              "type": "STRING",
+              "positions": [
+                0,
+                38,
+                12,
+                0,
+                0,
+                746,
+                11
+              ]
+            },
+            {
+              "entryId": 2,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3791,
+              "type": "STRING",
+              "positions": [
+                0,
+                78,
+                12,
+                0,
+                0,
+                1430,
+                95
+              ]
+            },
+            {
+              "entryId": 3,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3904,
+              "type": "STRING",
+              "positions": [
+                0,
+                118,
+                12,
+                0,
+                0,
+                2239,
+                23
+              ]
+            },
+            {
+              "entryId": 4,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3920,
+              "type": "STRING",
+              "positions": [
+                0,
+                158,
+                12,
+                0,
+                0,
+                2994,
+                17
+              ]
+            }
+          ],
+          "bloomFilterIndexes": [
+            {
+              "entryId": 0,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            },
+            {
+              "entryId": 1,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            },
+            {
+              "entryId": 2,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            },
+            {
+              "entryId": 3,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            },
+            {
+              "entryId": 4,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            }
+          ],
+          "stripeLevelBloomFilter": {
             "numHashFunctions": 4,
             "bitCount": 6272,
             "popCount": 138,
             "loadFactor": 0.022002551704645157,
             "expectedFpp": 2.3436470542037569E-7
           }
-        ],
-        "stripeLevelBloomFilter": {
-          "numHashFunctions": 4,
-          "bitCount": 6272,
-          "popCount": 138,
-          "loadFactor": 0.022002551704645157,
-          "expectedFpp": 2.3436470542037569E-7
         }
-      }]
+      ]
     },
     {
       "stripeNumber": 3,
@@ -841,150 +845,152 @@
           "dictionarySize": 35
         }
       ],
-      "indexes": [{
-        "columnId": 3,
-        "rowGroupIndexes": [
-          {
-            "entryId": 0,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3829,
-            "type": "STRING",
-            "positions": [
-              0,
-              0,
-              0,
-              0,
-              0,
-              0,
-              0
-            ]
-          },
-          {
-            "entryId": 1,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3853,
-            "type": "STRING",
-            "positions": [
-              0,
-              38,
-              12,
-              0,
-              0,
-              698,
-              74
-            ]
-          },
-          {
-            "entryId": 2,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3796,
-            "type": "STRING",
-            "positions": [
-              0,
-              78,
-              12,
-              0,
-              0,
-              1483,
-              39
-            ]
-          },
-          {
-            "entryId": 3,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3736,
-            "type": "STRING",
-            "positions": [
-              0,
-              118,
-              12,
-              0,
-              0,
-              2148,
-              155
-            ]
-          },
-          {
-            "entryId": 4,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3817,
-            "type": "STRING",
-            "positions": [
-              0,
-              158,
-              12,
-              0,
-              0,
-              3018,
-              8
-            ]
-          }
-        ],
-        "bloomFilterIndexes": [
-          {
-            "entryId": 0,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 1,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 2,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 3,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 4,
+      "indexes": [
+        {
+          "columnId": 3,
+          "rowGroupIndexes": [
+            {
+              "entryId": 0,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3829,
+              "type": "STRING",
+              "positions": [
+                0,
+                0,
+                0,
+                0,
+                0,
+                0,
+                0
+              ]
+            },
+            {
+              "entryId": 1,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3853,
+              "type": "STRING",
+              "positions": [
+                0,
+                38,
+                12,
+                0,
+                0,
+                698,
+                74
+              ]
+            },
+            {
+              "entryId": 2,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3796,
+              "type": "STRING",
+              "positions": [
+                0,
+                78,
+                12,
+                0,
+                0,
+                1483,
+                39
+              ]
+            },
+            {
+              "entryId": 3,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3736,
+              "type": "STRING",
+              "positions": [
+                0,
+                118,
+                12,
+                0,
+                0,
+                2148,
+                155
+              ]
+            },
+            {
+              "entryId": 4,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3817,
+              "type": "STRING",
+              "positions": [
+                0,
+                158,
+                12,
+                0,
+                0,
+                3018,
+                8
+              ]
+            }
+          ],
+          "bloomFilterIndexes": [
+            {
+              "entryId": 0,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            },
+            {
+              "entryId": 1,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            },
+            {
+              "entryId": 2,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            },
+            {
+              "entryId": 3,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            },
+            {
+              "entryId": 4,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            }
+          ],
+          "stripeLevelBloomFilter": {
             "numHashFunctions": 4,
             "bitCount": 6272,
             "popCount": 138,
             "loadFactor": 0.022002551704645157,
             "expectedFpp": 2.3436470542037569E-7
           }
-        ],
-        "stripeLevelBloomFilter": {
-          "numHashFunctions": 4,
-          "bitCount": 6272,
-          "popCount": 138,
-          "loadFactor": 0.022002551704645157,
-          "expectedFpp": 2.3436470542037569E-7
         }
-      }]
+      ]
     },
     {
       "stripeNumber": 4,
@@ -1082,150 +1088,152 @@
           "dictionarySize": 35
         }
       ],
-      "indexes": [{
-        "columnId": 3,
-        "rowGroupIndexes": [
-          {
-            "entryId": 0,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3959,
-            "type": "STRING",
-            "positions": [
-              0,
-              0,
-              0,
-              0,
-              0,
-              0,
-              0
-            ]
-          },
-          {
-            "entryId": 1,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3816,
-            "type": "STRING",
-            "positions": [
-              0,
-              38,
-              12,
-              0,
-              0,
-              495,
-              338
-            ]
-          },
-          {
-            "entryId": 2,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3883,
-            "type": "STRING",
-            "positions": [
-              0,
-              78,
-              12,
-              0,
-              0,
-              1449,
-              71
-            ]
-          },
-          {
-            "entryId": 3,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3938,
-            "type": "STRING",
-            "positions": [
-              0,
-              118,
-              12,
-              0,
-              0,
-              2207,
-              59
-            ]
-          },
-          {
-            "entryId": 4,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3863,
-            "type": "STRING",
-            "positions": [
-              0,
-              158,
-              12,
-              0,
-              0,
-              2838,
-              223
-            ]
-          }
-        ],
-        "bloomFilterIndexes": [
-          {
-            "entryId": 0,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 1,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 2,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 3,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 4,
+      "indexes": [
+        {
+          "columnId": 3,
+          "rowGroupIndexes": [
+            {
+              "entryId": 0,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3959,
+              "type": "STRING",
+              "positions": [
+                0,
+                0,
+                0,
+                0,
+                0,
+                0,
+                0
+              ]
+            },
+            {
+              "entryId": 1,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3816,
+              "type": "STRING",
+              "positions": [
+                0,
+                38,
+                12,
+                0,
+                0,
+                495,
+                338
+              ]
+            },
+            {
+              "entryId": 2,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3883,
+              "type": "STRING",
+              "positions": [
+                0,
+                78,
+                12,
+                0,
+                0,
+                1449,
+                71
+              ]
+            },
+            {
+              "entryId": 3,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3938,
+              "type": "STRING",
+              "positions": [
+                0,
+                118,
+                12,
+                0,
+                0,
+                2207,
+                59
+              ]
+            },
+            {
+              "entryId": 4,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3863,
+              "type": "STRING",
+              "positions": [
+                0,
+                158,
+                12,
+                0,
+                0,
+                2838,
+                223
+              ]
+            }
+          ],
+          "bloomFilterIndexes": [
+            {
+              "entryId": 0,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            },
+            {
+              "entryId": 1,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            },
+            {
+              "entryId": 2,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            },
+            {
+              "entryId": 3,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            },
+            {
+              "entryId": 4,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            }
+          ],
+          "stripeLevelBloomFilter": {
             "numHashFunctions": 4,
             "bitCount": 6272,
             "popCount": 138,
             "loadFactor": 0.022002551704645157,
             "expectedFpp": 2.3436470542037569E-7
           }
-        ],
-        "stripeLevelBloomFilter": {
-          "numHashFunctions": 4,
-          "bitCount": 6272,
-          "popCount": 138,
-          "loadFactor": 0.022002551704645157,
-          "expectedFpp": 2.3436470542037569E-7
         }
-      }]
+      ]
     },
     {
       "stripeNumber": 5,
@@ -1323,46 +1331,52 @@
           "dictionarySize": 35
         }
       ],
-      "indexes": [{
-        "columnId": 3,
-        "rowGroupIndexes": [{
-          "entryId": 0,
-          "count": 990,
-          "hasNull": true,
-          "min": "Darkness,",
-          "max": "worst",
-          "totalLength": 3963,
-          "type": "STRING",
-          "positions": [
-            0,
-            0,
-            0,
-            0,
-            0,
-            0,
-            0
-          ]
-        }],
-        "bloomFilterIndexes": [{
-          "entryId": 0,
-          "numHashFunctions": 4,
-          "bitCount": 6272,
-          "popCount": 138,
-          "loadFactor": 0.022002551704645157,
-          "expectedFpp": 2.3436470542037569E-7
-        }],
-        "stripeLevelBloomFilter": {
-          "numHashFunctions": 4,
-          "bitCount": 6272,
-          "popCount": 138,
-          "loadFactor": 0.022002551704645157,
-          "expectedFpp": 2.3436470542037569E-7
+      "indexes": [
+        {
+          "columnId": 3,
+          "rowGroupIndexes": [
+            {
+              "entryId": 0,
+              "count": 990,
+              "hasNull": true,
+              "min": "Darkness,",
+              "max": "worst",
+              "totalLength": 3963,
+              "type": "STRING",
+              "positions": [
+                0,
+                0,
+                0,
+                0,
+                0,
+                0,
+                0
+              ]
+            }
+          ],
+          "bloomFilterIndexes": [
+            {
+              "entryId": 0,
+              "numHashFunctions": 4,
+              "bitCount": 6272,
+              "popCount": 138,
+              "loadFactor": 0.022002551704645157,
+              "expectedFpp": 2.3436470542037569E-7
+            }
+          ],
+          "stripeLevelBloomFilter": {
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          }
         }
-      }]
+      ]
     }
   ],
   "fileLength": 272513,
   "paddingLength": 0,
-  "paddingRatio": 0,
+  "paddingRatio": 0.0,
   "status": "OK"
 }