You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by mm...@apache.org on 2016/07/24 08:23:21 UTC

[1/4] hive git commit: HIVE-14214: ORC Schema Evolution and Predicate Push Down do not work together (no rows returned) (Matt McCline, reviewed by Prasanth Jayachandran/Owen O'Malley)

Repository: hive
Updated Branches:
  refs/heads/master 9eb070ab7 -> b28ec7fdd


http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index bee859d..f356663 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -180,6 +180,10 @@ minitez.query.files.shared=acid_globallimit.q,\
   orc_merge_incompat1.q,\
   orc_merge_incompat2.q,\
   orc_merge_incompat3.q,\
+  orc_ppd_schema_evol_1a.q,\
+  orc_ppd_schema_evol_1b.q,\
+  orc_ppd_schema_evol_2a.q,\
+  orc_ppd_schema_evol_2b.q,\
   orc_vectorization_ppd.q,\
   parallel.q,\
   ptf.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/orc/src/java/org/apache/orc/impl/RecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/RecordReaderImpl.java b/orc/src/java/org/apache/orc/impl/RecordReaderImpl.java
index 23f532c..eb43ed6 100644
--- a/orc/src/java/org/apache/orc/impl/RecordReaderImpl.java
+++ b/orc/src/java/org/apache/orc/impl/RecordReaderImpl.java
@@ -76,6 +76,7 @@ public class RecordReaderImpl implements RecordReader {
   protected final TypeDescription schema;
   private final List<OrcProto.Type> types;
   private final int bufferSize;
+  private final SchemaEvolution evolution;
   private final boolean[] included;
   private final long rowIndexStride;
   private long rowInStripe = 0;
@@ -134,36 +135,50 @@ public class RecordReaderImpl implements RecordReader {
 
   protected RecordReaderImpl(ReaderImpl fileReader,
                              Reader.Options options) throws IOException {
-    SchemaEvolution treeReaderSchema;
     this.included = options.getInclude();
     included[0] = true;
     if (options.getSchema() == null) {
       if (LOG.isInfoEnabled()) {
-        LOG.info("Schema on read not provided -- using file schema " +
+        LOG.info("Reader schema not provided -- using file schema " +
             fileReader.getSchema());
       }
-      treeReaderSchema = new SchemaEvolution(fileReader.getSchema(), included);
+      evolution = new SchemaEvolution(fileReader.getSchema(), included);
     } else {
 
       // Now that we are creating a record reader for a file, validate that the schema to read
       // is compatible with the file schema.
       //
-      treeReaderSchema = new SchemaEvolution(fileReader.getSchema(),
+      evolution = new SchemaEvolution(fileReader.getSchema(),
           options.getSchema(),included);
+      if (LOG.isDebugEnabled() && evolution.hasConversion()) {
+        LOG.debug("ORC file " + fileReader.path.toString() +
+            " has data type conversion --\n" +
+            "reader schema: " + options.getSchema().toString() + "\n" +
+            "file schema:   " + fileReader.getSchema());
+      }
     }
-    this.schema = treeReaderSchema.getReaderSchema();
+    this.schema = evolution.getReaderSchema();
     this.path = fileReader.path;
     this.codec = fileReader.codec;
     this.types = fileReader.types;
     this.bufferSize = fileReader.bufferSize;
     this.rowIndexStride = fileReader.rowIndexStride;
     SearchArgument sarg = options.getSearchArgument();
-    if (sarg != null && rowIndexStride != 0) {
+    // We want to use the sarg for predicate evaluation but we have data type conversion
+    // (i.e Schema Evolution), so we currently ignore it.
+    if (sarg != null && rowIndexStride != 0 && !evolution.hasConversion()) {
       sargApp = new SargApplier(
           sarg, options.getColumnNames(), rowIndexStride, types,
           included.length);
     } else {
       sargApp = null;
+      if (evolution.hasConversion()) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(
+              "Skipping stripe elimination for {} since the schema has data type conversion",
+              fileReader.path);
+        }
+      }
     }
     long rows = 0;
     long skippedRows = 0;
@@ -205,8 +220,8 @@ public class RecordReaderImpl implements RecordReader {
       skipCorrupt = OrcConf.SKIP_CORRUPT_DATA.getBoolean(fileReader.conf);
     }
 
-    reader = TreeReaderFactory.createTreeReader(treeReaderSchema.getReaderSchema(),
-        treeReaderSchema, included, skipCorrupt);
+    reader = TreeReaderFactory.createTreeReader(evolution.getReaderSchema(),
+        evolution, included, skipCorrupt);
     indexes = new OrcProto.RowIndex[types.size()];
     bloomFilterIndices = new OrcProto.BloomFilterIndex[types.size()];
     advanceToNextRow(reader, 0L, true);

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/orc/src/java/org/apache/orc/impl/SchemaEvolution.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/SchemaEvolution.java b/orc/src/java/org/apache/orc/impl/SchemaEvolution.java
index 61a3f85..ce3af7a 100644
--- a/orc/src/java/org/apache/orc/impl/SchemaEvolution.java
+++ b/orc/src/java/org/apache/orc/impl/SchemaEvolution.java
@@ -20,9 +20,8 @@ package org.apache.orc.impl;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.Arrays;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -33,50 +32,64 @@ import org.apache.orc.TypeDescription;
  * has been schema evolution.
  */
 public class SchemaEvolution {
-  private final Map<Integer, TypeDescription> readerToFile;
+  private final TypeDescription[] readerFileTypes;
   private final boolean[] included;
   private final TypeDescription readerSchema;
+  private boolean hasConversion;
   private static final Log LOG = LogFactory.getLog(SchemaEvolution.class);
 
   public SchemaEvolution(TypeDescription readerSchema, boolean[] included) {
-    this.included = included;
-    readerToFile = null;
+    this.included = (included == null ? null : Arrays.copyOf(included, included.length));
     this.readerSchema = readerSchema;
+
+    hasConversion = false;
+
+    readerFileTypes = new TypeDescription[this.readerSchema.getMaximumId() + 1];
+    buildSameSchemaFileTypesArray();
   }
 
   public SchemaEvolution(TypeDescription fileSchema,
                          TypeDescription readerSchema,
                          boolean[] included) throws IOException {
-    readerToFile = new HashMap<>(readerSchema.getMaximumId() + 1);
-    this.included = included;
+    this.included = (included == null ? null : Arrays.copyOf(included, included.length));
     if (checkAcidSchema(fileSchema)) {
       this.readerSchema = createEventSchema(readerSchema);
     } else {
       this.readerSchema = readerSchema;
     }
-    buildMapping(fileSchema, this.readerSchema);
+
+    hasConversion = false;
+    readerFileTypes = new TypeDescription[this.readerSchema.getMaximumId() + 1];
+    buildConversionFileTypesArray(fileSchema, this.readerSchema);
   }
 
   public TypeDescription getReaderSchema() {
     return readerSchema;
   }
 
+  /**
+   * Is there Schema Evolution data type conversion?
+   * @return
+   */
+  public boolean hasConversion() {
+    return hasConversion;
+  }
+
   public TypeDescription getFileType(TypeDescription readerType) {
-    TypeDescription result;
-    if (readerToFile == null) {
-      if (included == null || included[readerType.getId()]) {
-        result = readerType;
-      } else {
-        result = null;
-      }
-    } else {
-      result = readerToFile.get(readerType.getId());
-    }
-    return result;
+    return getFileType(readerType.getId());
+  }
+
+  /**
+   * Get the file type by reader type id.
+   * @param readerType
+   * @return
+   */
+  public TypeDescription getFileType(int id) {
+    return readerFileTypes[id];
   }
 
-  void buildMapping(TypeDescription fileType,
-                    TypeDescription readerType) throws IOException {
+  void buildConversionFileTypesArray(TypeDescription fileType,
+                                     TypeDescription readerType) throws IOException {
     // if the column isn't included, don't map it
     if (included != null && !included[readerType.getId()]) {
       return;
@@ -101,9 +114,16 @@ public class SchemaEvolution {
         case CHAR:
         case VARCHAR:
           // We do conversion when same CHAR/VARCHAR type but different maxLength.
+          if (fileType.getMaxLength() != readerType.getMaxLength()) {
+            hasConversion = true;
+          }
           break;
         case DECIMAL:
           // We do conversion when same DECIMAL type but different precision/scale.
+          if (fileType.getPrecision() != readerType.getPrecision() ||
+              fileType.getScale() != readerType.getScale()) {
+            hasConversion = true;
+          }
           break;
         case UNION:
         case MAP:
@@ -113,7 +133,7 @@ public class SchemaEvolution {
           List<TypeDescription> readerChildren = readerType.getChildren();
           if (fileChildren.size() == readerChildren.size()) {
             for(int i=0; i < fileChildren.size(); ++i) {
-              buildMapping(fileChildren.get(i), readerChildren.get(i));
+              buildConversionFileTypesArray(fileChildren.get(i), readerChildren.get(i));
             }
           } else {
             isOk = false;
@@ -124,9 +144,12 @@ public class SchemaEvolution {
           // allow either side to have fewer fields than the other
           List<TypeDescription> fileChildren = fileType.getChildren();
           List<TypeDescription> readerChildren = readerType.getChildren();
+          if (fileChildren.size() != readerChildren.size()) {
+            hasConversion = true;
+          }
           int jointSize = Math.min(fileChildren.size(), readerChildren.size());
           for(int i=0; i < jointSize; ++i) {
-            buildMapping(fileChildren.get(i), readerChildren.get(i));
+            buildConversionFileTypesArray(fileChildren.get(i), readerChildren.get(i));
           }
           break;
         }
@@ -139,9 +162,14 @@ public class SchemaEvolution {
        */
 
       isOk = ConvertTreeReaderFactory.canConvert(fileType, readerType);
+      hasConversion = true;
     }
     if (isOk) {
-      readerToFile.put(readerType.getId(), fileType);
+      int id = readerType.getId();
+      if (readerFileTypes[id] != null) {
+        throw new RuntimeException("reader to file type entry already assigned");
+      }
+      readerFileTypes[id] = fileType;
     } else {
       throw new IOException(
           String.format(
@@ -151,6 +179,31 @@ public class SchemaEvolution {
     }
   }
 
+  /**
+   * Use to make a reader to file type array when the schema is the same.
+   * @return
+   */
+  private void buildSameSchemaFileTypesArray() {
+    buildSameSchemaFileTypesArrayRecurse(readerSchema);
+  }
+
+  void buildSameSchemaFileTypesArrayRecurse(TypeDescription readerType) {
+    if (included != null && !included[readerType.getId()]) {
+      return;
+    }
+    int id = readerType.getId();
+    if (readerFileTypes[id] != null) {
+      throw new RuntimeException("reader to file type entry already assigned");
+    }
+    readerFileTypes[id] = readerType;
+    List<TypeDescription> children = readerType.getChildren();
+    if (children != null) {
+      for (TypeDescription child : children) {
+        buildSameSchemaFileTypesArrayRecurse(child);
+      }
+    }
+  }
+
   private static boolean checkAcidSchema(TypeDescription type) {
     if (type.getCategory().equals(TypeDescription.Category.STRUCT)) {
       List<String> rootFields = type.getFieldNames();

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/orc/src/test/org/apache/orc/impl/TestSchemaEvolution.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/impl/TestSchemaEvolution.java b/orc/src/test/org/apache/orc/impl/TestSchemaEvolution.java
new file mode 100644
index 0000000..3cd0390
--- /dev/null
+++ b/orc/src/test/org/apache/orc/impl/TestSchemaEvolution.java
@@ -0,0 +1,107 @@
+/**
+ * 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.orc.impl;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.orc.TypeDescription;
+import org.junit.Test;
+
+public class TestSchemaEvolution {
+
+  @Test
+  public void testDataTypeConversion1() throws IOException {
+    TypeDescription fileStruct1 = TypeDescription.createStruct()
+        .addField("f1", TypeDescription.createInt())
+        .addField("f2", TypeDescription.createString())
+        .addField("f3", TypeDescription.createDecimal().withPrecision(38).withScale(10));
+    SchemaEvolution same1 = new SchemaEvolution(fileStruct1, null);
+    assertFalse(same1.hasConversion());
+    TypeDescription readerStruct1 = TypeDescription.createStruct()
+        .addField("f1", TypeDescription.createInt())
+        .addField("f2", TypeDescription.createString())
+        .addField("f3", TypeDescription.createDecimal().withPrecision(38).withScale(10));
+    SchemaEvolution both1 = new SchemaEvolution(fileStruct1, readerStruct1, null);
+    assertFalse(both1.hasConversion());
+    TypeDescription readerStruct1diff = TypeDescription.createStruct()
+        .addField("f1", TypeDescription.createLong())
+        .addField("f2", TypeDescription.createString())
+        .addField("f3", TypeDescription.createDecimal().withPrecision(38).withScale(10));
+    SchemaEvolution both1diff = new SchemaEvolution(fileStruct1, readerStruct1diff, null);
+    assertTrue(both1diff.hasConversion());
+    TypeDescription readerStruct1diffPrecision = TypeDescription.createStruct()
+        .addField("f1", TypeDescription.createInt())
+        .addField("f2", TypeDescription.createString())
+        .addField("f3", TypeDescription.createDecimal().withPrecision(12).withScale(10));
+    SchemaEvolution both1diffPrecision = new SchemaEvolution(fileStruct1, readerStruct1diffPrecision, null);
+    assertTrue(both1diffPrecision.hasConversion());
+  }
+
+  @Test
+  public void testDataTypeConversion2() throws IOException {
+    TypeDescription fileStruct2 = TypeDescription.createStruct()
+        .addField("f1", TypeDescription.createUnion()
+            .addUnionChild(TypeDescription.createByte())
+            .addUnionChild(TypeDescription.createDecimal()
+                .withPrecision(20).withScale(10)))
+        .addField("f2", TypeDescription.createStruct()
+            .addField("f3", TypeDescription.createDate())
+            .addField("f4", TypeDescription.createDouble())
+            .addField("f5", TypeDescription.createBoolean()))
+        .addField("f6", TypeDescription.createChar().withMaxLength(100));
+    SchemaEvolution same2 = new SchemaEvolution(fileStruct2, null);
+    assertFalse(same2.hasConversion());
+    TypeDescription readerStruct2 = TypeDescription.createStruct()
+        .addField("f1", TypeDescription.createUnion()
+            .addUnionChild(TypeDescription.createByte())
+            .addUnionChild(TypeDescription.createDecimal()
+                .withPrecision(20).withScale(10)))
+        .addField("f2", TypeDescription.createStruct()
+            .addField("f3", TypeDescription.createDate())
+            .addField("f4", TypeDescription.createDouble())
+            .addField("f5", TypeDescription.createBoolean()))
+        .addField("f6", TypeDescription.createChar().withMaxLength(100));
+    SchemaEvolution both2 = new SchemaEvolution(fileStruct2, readerStruct2, null);
+    assertFalse(both2.hasConversion());
+    TypeDescription readerStruct2diff = TypeDescription.createStruct()
+        .addField("f1", TypeDescription.createUnion()
+            .addUnionChild(TypeDescription.createByte())
+            .addUnionChild(TypeDescription.createDecimal()
+                .withPrecision(20).withScale(10)))
+        .addField("f2", TypeDescription.createStruct()
+            .addField("f3", TypeDescription.createDate())
+            .addField("f4", TypeDescription.createDouble()))
+        .addField("f6", TypeDescription.createChar().withMaxLength(100));
+    SchemaEvolution both2diff = new SchemaEvolution(fileStruct2, readerStruct2diff, null);
+    assertTrue(both2diff.hasConversion());
+    TypeDescription readerStruct2diffChar = TypeDescription.createStruct()
+        .addField("f1", TypeDescription.createUnion()
+            .addUnionChild(TypeDescription.createByte())
+            .addUnionChild(TypeDescription.createDecimal()
+                .withPrecision(20).withScale(10)))
+        .addField("f2", TypeDescription.createStruct()
+            .addField("f3", TypeDescription.createDate())
+            .addField("f4", TypeDescription.createDouble()))
+        .addField("f6", TypeDescription.createChar().withMaxLength(80));
+    SchemaEvolution both2diffChar = new SchemaEvolution(fileStruct2, readerStruct2diffChar, null);
+    assertTrue(both2diffChar.hasConversion());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
index 69d58d6..63d02fb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.ql.io.orc;
 
 import org.apache.orc.impl.InStream;
+import org.apache.orc.impl.SchemaEvolution;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -98,6 +99,7 @@ import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
+import org.apache.hadoop.hive.ql.plan.MapWork;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
 import org.apache.hadoop.hive.serde2.SerDeStats;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -525,16 +527,17 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
     private SplitStrategyKind splitStrategyKind;
     private final SearchArgument sarg;
 
-    Context(Configuration conf) {
+    Context(Configuration conf) throws IOException {
       this(conf, 1, null);
     }
 
-    Context(Configuration conf, final int minSplits) {
+    Context(Configuration conf, final int minSplits) throws IOException {
       this(conf, minSplits, null);
     }
 
     @VisibleForTesting
-    Context(Configuration conf, final int minSplits, ExternalFooterCachesByConf efc) {
+    Context(Configuration conf, final int minSplits, ExternalFooterCachesByConf efc)
+        throws IOException {
       this.conf = conf;
       this.forceThreadpool = HiveConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST);
       this.sarg = ConvertAstToSearchArg.createFromConf(conf);
@@ -579,6 +582,13 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
           // using such an aggregate fileId cache is not bulletproof and should be disable-able.
           boolean useExternalCache = HiveConf.getBoolVar(
               conf, HiveConf.ConfVars.HIVE_ORC_MS_FOOTER_CACHE_ENABLED);
+          if (useExternalCache) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(
+                "Turning off hive.orc.splits.ms.footer.cache.enabled since it is not fully supported yet");
+            }
+            useExternalCache = false;
+          }
           if (localCache == null) {
             localCache = new LocalCache(numThreads, cacheStripeDetailsSize, useSoftReference);
           }
@@ -655,19 +665,21 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
     private final FileSystem fs;
     private final HdfsFileStatusWithId fileWithId;
     private final OrcTail orcTail;
+    private final List<OrcProto.Type> readerTypes;
     private final boolean isOriginal;
     private final List<DeltaMetaData> deltas;
     private final boolean hasBase;
     private final ByteBuffer ppdResult;
 
     SplitInfo(Context context, FileSystem fs, HdfsFileStatusWithId fileWithId, OrcTail orcTail,
-        boolean isOriginal, List<DeltaMetaData> deltas, boolean hasBase, Path dir,
-        boolean[] covered, ByteBuffer ppdResult) throws IOException {
+        List<OrcProto.Type> readerTypes, boolean isOriginal, List<DeltaMetaData> deltas,
+        boolean hasBase, Path dir, boolean[] covered, ByteBuffer ppdResult) throws IOException {
       super(dir, context.numBuckets, deltas, covered);
       this.context = context;
       this.fs = fs;
       this.fileWithId = fileWithId;
       this.orcTail = orcTail;
+      this.readerTypes = readerTypes;
       this.isOriginal = isOriginal;
       this.deltas = deltas;
       this.hasBase = hasBase;
@@ -676,10 +688,10 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
 
     @VisibleForTesting
     public SplitInfo(Context context, FileSystem fs, FileStatus fileStatus, OrcTail orcTail,
-        boolean isOriginal, ArrayList<DeltaMetaData> deltas, boolean hasBase, Path dir,
-        boolean[] covered) throws IOException {
+        List<OrcProto.Type> readerTypes,  boolean isOriginal, ArrayList<DeltaMetaData> deltas,
+        boolean hasBase, Path dir, boolean[] covered) throws IOException {
       this(context, fs, AcidUtils.createOriginalObj(null, fileStatus),
-          orcTail, isOriginal, deltas, hasBase, dir, covered, null);
+          orcTail, readerTypes, isOriginal, deltas, hasBase, dir, covered, null);
     }
   }
 
@@ -707,6 +719,7 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
     private final List<DeltaMetaData> deltas;
     private final boolean[] covered;
     final boolean isOriginal;
+    final List<OrcProto.Type> readerTypes;
     // References to external fields for async SplitInfo generation.
     private List<Future<List<OrcSplit>>> splitFuturesRef = null;
     private List<OrcSplit> splitsRef = null;
@@ -714,13 +727,14 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
     private final boolean allowSyntheticFileIds;
 
     public ETLSplitStrategy(Context context, FileSystem fs, Path dir,
-        List<HdfsFileStatusWithId> children, boolean isOriginal, List<DeltaMetaData> deltas,
-        boolean[] covered, UserGroupInformation ugi, boolean allowSyntheticFileIds) {
+        List<HdfsFileStatusWithId> children, List<OrcProto.Type> readerTypes, boolean isOriginal,
+        List<DeltaMetaData> deltas, boolean[] covered, UserGroupInformation ugi, boolean allowSyntheticFileIds) {
       assert !children.isEmpty();
       this.context = context;
       this.dirs = Lists.newArrayList(new ETLDir(dir, fs, children.size()));
       this.files = children;
       this.isOriginal = isOriginal;
+      this.readerTypes = readerTypes;
       this.deltas = deltas;
       this.covered = covered;
       this.ugi = ugi;
@@ -760,7 +774,7 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
           }
           // Ignore files eliminated by PPD, or of 0 length.
           if (ppdResult != FooterCache.NO_SPLIT_AFTER_PPD && file.getFileStatus().getLen() > 0) {
-            result.add(new SplitInfo(context, dir.fs, file, orcTail,
+            result.add(new SplitInfo(context, dir.fs, file, orcTail, readerTypes,
                 isOriginal, deltas, true, dir.dir, covered, ppdResult));
           }
         }
@@ -774,7 +788,7 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
           }
           // ignore files of 0 length
           if (file.getFileStatus().getLen() > 0) {
-            result.add(new SplitInfo(context, dir.fs, file, null,
+            result.add(new SplitInfo(context, dir.fs, file, null, readerTypes,
                 isOriginal, deltas, true, dir.dir, covered, null));
           }
         }
@@ -1061,10 +1075,11 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
     private final long blockSize;
     private final TreeMap<Long, BlockLocation> locations;
     private OrcTail orcTail;
+    private final List<OrcProto.Type> readerTypes;
     private List<StripeInformation> stripes;
     private List<StripeStatistics> stripeStats;
-    private List<OrcProto.Type> types;
-    private boolean[] includedCols;
+    private List<OrcProto.Type> fileTypes;
+    private boolean[] readerIncluded;
     private final boolean isOriginal;
     private final List<DeltaMetaData> deltas;
     private final boolean hasBase;
@@ -1074,6 +1089,7 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
     private final ByteBuffer ppdResult;
     private final UserGroupInformation ugi;
     private final boolean allowSyntheticFileIds;
+    private SchemaEvolution evolution;
 
     public SplitGenerator(SplitInfo splitInfo, UserGroupInformation ugi,
         boolean allowSyntheticFileIds) throws IOException {
@@ -1084,6 +1100,7 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
       this.fsFileId = splitInfo.fileWithId.getFileId();
       this.blockSize = this.file.getBlockSize();
       this.orcTail = splitInfo.orcTail;
+      this.readerTypes = splitInfo.readerTypes;
       // TODO: potential DFS call
       this.locations = SHIMS.getLocationsWithOffset(fs, file);
       this.isOriginal = splitInfo.isOriginal;
@@ -1255,12 +1272,24 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
         // We can't eliminate stripes if there are deltas because the
         // deltas may change the rows making them match the predicate.
         if ((deltas == null || deltas.isEmpty()) && context.sarg != null) {
-          String[] colNames = extractNeededColNames(types, context.conf, includedCols, isOriginal);
-          if (colNames == null) {
-            LOG.warn("Skipping split elimination for {} as column names is null", file.getPath());
+          // Also, we currently do not use predicate evaluation when the schema has data type
+          // conversion.
+          if (evolution.hasConversion()) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(
+                  "Skipping split elimination for {} since the schema has data type conversion",
+                  file.getPath());
+            }
           } else {
-            includeStripe = pickStripes(context.sarg, colNames, writerVersion, isOriginal,
-                stripeStats, stripes.size(), file.getPath());
+            String[] colNames =
+                extractNeededColNames((readerTypes == null ? fileTypes : readerTypes),
+                    context.conf, readerIncluded, isOriginal);
+            if (colNames == null) {
+              LOG.warn("Skipping split elimination for {} as column names is null", file.getPath());
+            } else {
+              includeStripe = pickStripes(context.sarg, colNames, writerVersion, isOriginal,
+                  stripeStats, stripes.size(), file.getPath());
+            }
           }
         }
         return generateSplitsFromStripes(includeStripe);
@@ -1374,29 +1403,67 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
       }
       stripes = orcTail.getStripes();
       stripeStats = orcTail.getStripeStatistics();
-      types = orcTail.getTypes();
+      fileTypes = orcTail.getTypes();
+      TypeDescription fileSchema = OrcUtils.convertTypeFromProtobuf(fileTypes, 0);
+      if (readerTypes == null) {
+        readerIncluded = genIncludedColumns(fileTypes, context.conf, isOriginal);
+        evolution = new SchemaEvolution(fileSchema, readerIncluded);
+      } else {
+        // The readerSchema always comes in without ACID columns.
+        readerIncluded = genIncludedColumns(readerTypes, context.conf, /* isOriginal */ true);
+        if (readerIncluded != null && !isOriginal) {
+          // We shift the include columns here because the SchemaEvolution constructor will
+          // add the ACID event metadata the readerSchema...
+          readerIncluded = shiftReaderIncludedForAcid(readerIncluded);
+        }
+        TypeDescription readerSchema = OrcUtils.convertTypeFromProtobuf(readerTypes, 0);
+        evolution = new SchemaEvolution(fileSchema, readerSchema, readerIncluded);
+      }
       writerVersion = orcTail.getWriterVersion();
-      includedCols = genIncludedColumns(types, context.conf, isOriginal);
       List<OrcProto.ColumnStatistics> fileColStats = orcTail.getFooter().getStatisticsList();
-      projColsUncompressedSize = computeProjectionSize(types, fileColStats, includedCols,
+      boolean[] fileIncluded;
+      if (readerTypes == null) {
+        fileIncluded = readerIncluded;
+      } else {
+        fileIncluded = new boolean[fileTypes.size()];
+        final int readerSchemaSize = readerTypes.size();
+        for (int i = 0; i < readerSchemaSize; i++) {
+          TypeDescription fileType = evolution.getFileType(i);
+          if (fileType != null) {
+            fileIncluded[fileType.getId()] = true;
+          }
+        }
+      }
+      projColsUncompressedSize = computeProjectionSize(fileTypes, fileColStats, fileIncluded,
           isOriginal);
       if (!context.footerInSplits) {
         orcTail = null;
       }
     }
 
-    private long computeProjectionSize(List<OrcProto.Type> types,
-        List<OrcProto.ColumnStatistics> stats, boolean[] includedCols, boolean isOriginal) {
+    private long computeProjectionSize(List<OrcProto.Type> fileTypes,
+        List<OrcProto.ColumnStatistics> stats, boolean[] fileIncluded, boolean isOriginal) {
       final int rootIdx = getRootColumn(isOriginal);
       List<Integer> internalColIds = Lists.newArrayList();
-      if (includedCols != null) {
-        for (int i = 0; i < includedCols.length; i++) {
-          if (includedCols[i]) {
+      if (fileIncluded != null) {
+        for (int i = 0; i < fileIncluded.length; i++) {
+          if (fileIncluded[i]) {
             internalColIds.add(rootIdx + i);
           }
         }
       }
-      return ReaderImpl.getRawDataSizeFromColIndices(internalColIds, types, stats);
+      return ReaderImpl.getRawDataSizeFromColIndices(internalColIds, fileTypes, stats);
+    }
+
+    private boolean[] shiftReaderIncludedForAcid(boolean[] included) {
+      // We always need the base row
+      included[0] = true;
+      boolean[] newIncluded = new boolean[included.length + OrcRecordUpdater.FIELDS];
+      Arrays.fill(newIncluded, 0, OrcRecordUpdater.FIELDS, true);
+      for(int i= 0; i < included.length; ++i) {
+        newIncluded[i + OrcRecordUpdater.FIELDS] = included[i];
+      }
+      return newIncluded;
     }
   }
 
@@ -1431,6 +1498,21 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
       pathFutures.add(ecs.submit(fileGenerator));
     }
 
+    boolean isTransactionalTableScan =
+        HiveConf.getBoolVar(conf, ConfVars.HIVE_TRANSACTIONAL_TABLE_SCAN);
+    boolean isSchemaEvolution = HiveConf.getBoolVar(conf, ConfVars.HIVE_SCHEMA_EVOLUTION);
+    TypeDescription readerSchema =
+        OrcInputFormat.getDesiredRowTypeDescr(conf, isTransactionalTableScan, Integer.MAX_VALUE);
+    List<OrcProto.Type> readerTypes = null;
+    if (readerSchema != null) {
+      readerTypes = OrcUtils.getOrcTypes(readerSchema);
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Generate splits schema evolution property " + isSchemaEvolution +
+        " reader schema " + (readerSchema == null ? "NULL" : readerSchema.toString()) +
+        " transactional scan property " + isTransactionalTableScan);
+    }
+
     // complete path futures and schedule split generation
     try {
       CombinedCtx combinedCtx = (context.splitStrategyBatchMs > 0) ? new CombinedCtx() : null;
@@ -1459,7 +1541,8 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
         // We have received a new directory information, make a split strategy.
         --resultsLeft;
         SplitStrategy<?> splitStrategy = determineSplitStrategy(combinedCtx, context, adi.fs,
-            adi.splitPath, adi.acidInfo, adi.baseOrOriginalFiles, ugi, allowSyntheticFileIds);
+            adi.splitPath, adi.acidInfo, adi.baseOrOriginalFiles, readerTypes, ugi,
+            allowSyntheticFileIds);
         if (splitStrategy == null) continue; // Combined.
 
         if (isDebugEnabled) {
@@ -1516,7 +1599,7 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
   @VisibleForTesting
   // We could have this as a protected method w/no class, but half of Hive is static, so there.
   public static class ContextFactory {
-    public Context create(Configuration conf, int numSplits) {
+    public Context create(Configuration conf, int numSplits) throws IOException {
       return new Context(conf, numSplits);
     }
   }
@@ -1537,14 +1620,16 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
 
   private static SplitStrategy<?> combineOrCreateETLStrategy(CombinedCtx combinedCtx,
       Context context, FileSystem fs, Path dir, List<HdfsFileStatusWithId> files,
-      List<DeltaMetaData> deltas, boolean[] covered, boolean isOriginal,
-      UserGroupInformation ugi, boolean allowSyntheticFileIds) {
+      List<DeltaMetaData> deltas, boolean[] covered, List<OrcProto.Type> readerTypes,
+      boolean isOriginal, UserGroupInformation ugi, boolean allowSyntheticFileIds) {
     if (!deltas.isEmpty() || combinedCtx == null) {
       return new ETLSplitStrategy(
-          context, fs, dir, files, isOriginal, deltas, covered, ugi, allowSyntheticFileIds);
+          context, fs, dir, files, readerTypes, isOriginal, deltas, covered, ugi,
+          allowSyntheticFileIds);
     } else if (combinedCtx.combined == null) {
       combinedCtx.combined = new ETLSplitStrategy(
-          context, fs, dir, files, isOriginal, deltas, covered, ugi, allowSyntheticFileIds);
+          context, fs, dir, files, readerTypes, isOriginal, deltas, covered, ugi,
+          allowSyntheticFileIds);
       combinedCtx.combineStartUs = System.nanoTime();
       return null;
     } else {
@@ -1554,11 +1639,13 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
       case YES: return null;
       case NO_AND_CONTINUE:
         return new ETLSplitStrategy(
-            context, fs, dir, files, isOriginal, deltas, covered, ugi, allowSyntheticFileIds);
+            context, fs, dir, files, readerTypes, isOriginal, deltas, covered, ugi,
+            allowSyntheticFileIds);
       case NO_AND_SWAP: {
         ETLSplitStrategy oldBase = combinedCtx.combined;
         combinedCtx.combined = new ETLSplitStrategy(
-            context, fs, dir, files, isOriginal, deltas, covered, ugi, allowSyntheticFileIds);
+            context, fs, dir, files, readerTypes, isOriginal, deltas, covered, ugi,
+            allowSyntheticFileIds);
         combinedCtx.combineStartUs = System.nanoTime();
         return oldBase;
       }
@@ -1868,8 +1955,8 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
   @VisibleForTesting
   static SplitStrategy<?> determineSplitStrategy(CombinedCtx combinedCtx, Context context,
       FileSystem fs, Path dir, AcidUtils.Directory dirInfo,
-      List<HdfsFileStatusWithId> baseOrOriginalFiles, UserGroupInformation ugi,
-      boolean allowSyntheticFileIds) {
+      List<HdfsFileStatusWithId> baseOrOriginalFiles, List<OrcProto.Type> readerTypes,
+      UserGroupInformation ugi, boolean allowSyntheticFileIds) {
     Path base = dirInfo.getBaseDirectory();
     List<HdfsFileStatusWithId> original = dirInfo.getOriginalFiles();
     List<DeltaMetaData> deltas = AcidUtils.serializeDeltas(dirInfo.getCurrentDirectories());
@@ -1902,12 +1989,12 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
         case ETL:
           // ETL strategy requested through config
           return combineOrCreateETLStrategy(combinedCtx, context, fs, dir, baseOrOriginalFiles,
-              deltas, covered, isOriginal, ugi, allowSyntheticFileIds);
+              deltas, covered, readerTypes, isOriginal, ugi, allowSyntheticFileIds);
         default:
           // HYBRID strategy
           if (avgFileSize > context.maxSize || totalFiles <= context.etlFileThreshold) {
             return combineOrCreateETLStrategy(combinedCtx, context, fs, dir, baseOrOriginalFiles,
-                deltas, covered, isOriginal, ugi, allowSyntheticFileIds);
+                deltas, covered, readerTypes, isOriginal, ugi, allowSyntheticFileIds);
           } else {
             return new BISplitStrategy(context, fs, dir, baseOrOriginalFiles,
                 isOriginal, deltas, covered, allowSyntheticFileIds);

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcNewInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcNewInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcNewInputFormat.java
index c4a7226..3b05fb2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcNewInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcNewInputFormat.java
@@ -136,7 +136,7 @@ public class OrcNewInputFormat extends InputFormat<NullWritable, OrcStruct>{
   }
 
   // Nearly C/P from OrcInputFormat; there are too many statics everywhere to sort this out.
-  private Context createContext(Configuration conf, int numSplits) {
+  private Context createContext(Configuration conf, int numSplits) throws IOException {
     // Use threads to resolve directories into splits.
     if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_ORC_MS_FOOTER_CACHE_ENABLED)) {
       // Create HiveConf once, since this is expensive.

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
index a001f8e..a675a34 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
@@ -768,7 +768,7 @@ public class TestInputOutputFormat {
       MockFileSystem fs, String path, OrcInputFormat.CombinedCtx combineCtx) throws IOException {
     OrcInputFormat.AcidDirInfo adi = createAdi(context, fs, path);
     return OrcInputFormat.determineSplitStrategy(combineCtx, context,
-        adi.fs, adi.splitPath, adi.acidInfo, adi.baseOrOriginalFiles, null, true);
+        adi.fs, adi.splitPath, adi.acidInfo, adi.baseOrOriginalFiles, null, null, true);
   }
 
   public OrcInputFormat.AcidDirInfo createAdi(
@@ -781,7 +781,7 @@ public class TestInputOutputFormat {
       OrcInputFormat.Context context, OrcInputFormat.FileGenerator gen) throws IOException {
     OrcInputFormat.AcidDirInfo adi = gen.call();
     return OrcInputFormat.determineSplitStrategy(
-        null, context, adi.fs, adi.splitPath, adi.acidInfo, adi.baseOrOriginalFiles, null, true);
+        null, context, adi.fs, adi.splitPath, adi.acidInfo, adi.baseOrOriginalFiles, null, null, true);
   }
 
   public static class MockBlock {
@@ -1389,7 +1389,7 @@ public class TestInputOutputFormat {
     OrcInputFormat.Context context = new OrcInputFormat.Context(conf);
     OrcInputFormat.SplitGenerator splitter =
         new OrcInputFormat.SplitGenerator(new OrcInputFormat.SplitInfo(context, fs,
-            fs.getFileStatus(new Path("/a/file")), null, true,
+            fs.getFileStatus(new Path("/a/file")), null, null, true,
             new ArrayList<AcidInputFormat.DeltaMetaData>(), true, null, null), null, true);
     OrcSplit result = splitter.createSplit(0, 200, null);
     assertEquals(0, result.getStart());
@@ -1430,7 +1430,7 @@ public class TestInputOutputFormat {
     OrcInputFormat.Context context = new OrcInputFormat.Context(conf);
     OrcInputFormat.SplitGenerator splitter =
         new OrcInputFormat.SplitGenerator(new OrcInputFormat.SplitInfo(context, fs,
-            fs.getFileStatus(new Path("/a/file")), null, true,
+            fs.getFileStatus(new Path("/a/file")), null, null, true,
             new ArrayList<AcidInputFormat.DeltaMetaData>(), true, null, null), null, true);
     List<OrcSplit> results = splitter.call();
     OrcSplit result = results.get(0);
@@ -1453,7 +1453,7 @@ public class TestInputOutputFormat {
     HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, 0);
     context = new OrcInputFormat.Context(conf);
     splitter = new OrcInputFormat.SplitGenerator(new OrcInputFormat.SplitInfo(context, fs,
-      fs.getFileStatus(new Path("/a/file")), null, true,
+      fs.getFileStatus(new Path("/a/file")), null, null, true,
         new ArrayList<AcidInputFormat.DeltaMetaData>(), true, null, null), null, true);
     results = splitter.call();
     for(int i=0; i < stripeSizes.length; ++i) {
@@ -1481,7 +1481,7 @@ public class TestInputOutputFormat {
     OrcInputFormat.Context context = new OrcInputFormat.Context(conf);
     OrcInputFormat.SplitGenerator splitter =
         new OrcInputFormat.SplitGenerator(new OrcInputFormat.SplitInfo(context, fs,
-            fs.getFileStatus(new Path("/a/file")), null, true,
+            fs.getFileStatus(new Path("/a/file")), null, null, true,
             new ArrayList<AcidInputFormat.DeltaMetaData>(), true, null, null), null, true);
     List<OrcSplit> results = splitter.call();
     OrcSplit result = results.get(0);
@@ -1503,7 +1503,7 @@ public class TestInputOutputFormat {
     HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, 0);
     context = new OrcInputFormat.Context(conf);
     splitter = new OrcInputFormat.SplitGenerator(new OrcInputFormat.SplitInfo(context, fs,
-        fs.getFileStatus(new Path("/a/file")), null, true,
+        fs.getFileStatus(new Path("/a/file")), null, null, true,
         new ArrayList<AcidInputFormat.DeltaMetaData>(),
         true, null, null), null, true);
     results = splitter.call();
@@ -1523,7 +1523,7 @@ public class TestInputOutputFormat {
     HiveConf.setLongVar(conf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, 100000);
     context = new OrcInputFormat.Context(conf);
     splitter = new OrcInputFormat.SplitGenerator(new OrcInputFormat.SplitInfo(context, fs,
-        fs.getFileStatus(new Path("/a/file")), null, true,
+        fs.getFileStatus(new Path("/a/file")), null, null, true,
         new ArrayList<AcidInputFormat.DeltaMetaData>(),
         true, null, null), null, true);
     results = splitter.call();

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcSplitElimination.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcSplitElimination.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcSplitElimination.java
index 62a0ab0..b26401d 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcSplitElimination.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcSplitElimination.java
@@ -64,6 +64,7 @@ import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
@@ -536,11 +537,13 @@ public class TestOrcSplitElimination {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestOrcSplitElimination.class);
 
+  @Ignore("External cache has been turned off for now")
   @Test
   public void testExternalFooterCache() throws Exception {
     testFooterExternalCacheImpl(false);
   }
 
+  @Ignore("External cache has been turned off for now")
   @Test
   public void testExternalFooterCachePpd() throws Exception {
     testFooterExternalCacheImpl(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/ql/src/test/queries/clientpositive/orc_ppd_schema_evol_1a.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/orc_ppd_schema_evol_1a.q b/ql/src/test/queries/clientpositive/orc_ppd_schema_evol_1a.q
new file mode 100644
index 0000000..f52c2ac
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/orc_ppd_schema_evol_1a.q
@@ -0,0 +1,42 @@
+set hive.cli.print.header=true;
+set hive.metastore.disallow.incompatible.col.type.changes=true;
+set hive.optimize.ppd=false;
+set hive.optimize.index.filter=false;
+set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+
+
+create table unique_1( 
+i int, 
+d double, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile;
+
+load data local inpath '../../data/files/unique_1.txt' into table unique_1;
+
+create table test1 stored as orc as select * from unique_1;
+
+SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecTezSummaryPrinter;
+
+alter table test1 change column i i string;
+
+set hive.optimize.ppd=false;
+set hive.optimize.index.filter=false;
+set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+
+select s from test1 where i = '-1591211872';
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+
+select s from test1 where i = -1591211872;
+
+set hive.optimize.ppd=true;
+set hive.optimize.index.filter=true;
+set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+
+select s from test1 where i = '-1591211872';
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+
+select s from test1 where i = -1591211872;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/ql/src/test/queries/clientpositive/orc_ppd_schema_evol_1b.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/orc_ppd_schema_evol_1b.q b/ql/src/test/queries/clientpositive/orc_ppd_schema_evol_1b.q
new file mode 100644
index 0000000..e669f06
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/orc_ppd_schema_evol_1b.q
@@ -0,0 +1,60 @@
+set hive.cli.print.header=true;
+set hive.metastore.disallow.incompatible.col.type.changes=true;
+set hive.optimize.ppd=false;
+set hive.optimize.index.filter=false;
+set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+
+create table unique_1( 
+i int, 
+d double, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile;
+
+load data local inpath '../../data/files/unique_1.txt' into table unique_1;
+
+create table unique_2( 
+i int, 
+d double, 
+s string)
+row format delimited 
+fields terminated by '|' 
+stored as textfile;
+
+load data local inpath '../../data/files/unique_2.txt' into table unique_2;
+
+create table test_two_files( 
+i int, 
+d double, 
+s string)
+stored as orc;
+
+insert into table test_two_files select * from unique_1 where i <= 0;
+insert into table test_two_files select * from unique_2 where i > 0;
+
+SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecTezSummaryPrinter;
+
+alter table test_two_files change column i i string;
+
+set hive.optimize.ppd=false;
+set hive.optimize.index.filter=false;
+set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+
+select s from test_two_files where i = -1591211872;
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+
+select s from test_two_files where i = -1591211872;
+
+
+set hive.optimize.ppd=true;
+set hive.optimize.index.filter=true;
+set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+
+select s from test_two_files where i = '-1591211872';
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+
+select s from test_two_files where i = -1591211872;
+

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/ql/src/test/queries/clientpositive/orc_ppd_schema_evol_2a.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/orc_ppd_schema_evol_2a.q b/ql/src/test/queries/clientpositive/orc_ppd_schema_evol_2a.q
new file mode 100644
index 0000000..609924d
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/orc_ppd_schema_evol_2a.q
@@ -0,0 +1,43 @@
+set hive.cli.print.header=true;
+set hive.metastore.disallow.incompatible.col.type.changes=false;
+set hive.optimize.ppd=false;
+set hive.optimize.index.filter=false;
+set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+
+
+
+create table unique_1( 
+i int, 
+d string, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile;
+
+load data local inpath '../../data/files/unique_1.txt' into table unique_1;
+
+create table test1 stored as orc as select * from unique_1 order by d;
+
+SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecTezSummaryPrinter;
+
+alter table test1 change column d d double;
+
+set hive.optimize.ppd=false;
+set hive.optimize.index.filter=false;
+set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+
+select s from test1 where d = -4996703.42;
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+
+select s from test1 where d = -4996703.42;
+
+set hive.optimize.ppd=true;
+set hive.optimize.index.filter=true;
+set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+
+select s from test1 where d = -4996703.42;
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+
+select s from test1 where d = -4996703.42;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/ql/src/test/queries/clientpositive/orc_ppd_schema_evol_2b.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/orc_ppd_schema_evol_2b.q b/ql/src/test/queries/clientpositive/orc_ppd_schema_evol_2b.q
new file mode 100644
index 0000000..edebeef
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/orc_ppd_schema_evol_2b.q
@@ -0,0 +1,62 @@
+set hive.cli.print.header=true;
+set hive.metastore.disallow.incompatible.col.type.changes=false;
+set hive.optimize.ppd=false;
+set hive.optimize.index.filter=false;
+set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+
+
+create table unique_1( 
+i int, 
+d string, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile;
+
+load data local inpath '../../data/files/unique_1.txt' into table unique_1;
+
+create table unique_2( 
+i int, 
+d string, 
+s string)
+row format delimited 
+fields terminated by '|' 
+stored as textfile;
+
+load data local inpath '../../data/files/unique_2.txt' into table unique_2;
+
+create table test_two_files( 
+i int, 
+d string, 
+s string)
+stored as orc;
+
+insert into table test_two_files select * from unique_1 where cast(d as double) <= 0 order by cast(d as double);
+insert into table test_two_files select * from unique_2 where cast(d as double) > 0 order by cast(d as double);
+
+SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecTezSummaryPrinter;
+
+alter table test_two_files change column d d double;
+
+set hive.optimize.ppd=false;
+set hive.optimize.index.filter=false;
+set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+
+select s from test_two_files where d = -4996703.42;
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+
+select s from test_two_files where d = -4996703.42;
+
+
+set hive.optimize.ppd=true;
+set hive.optimize.index.filter=true;
+set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+
+select s from test_two_files where d = -4996703.42;
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+
+select s from test_two_files where d = -4996703.42;
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/ql/src/test/results/clientpositive/orc_ppd_schema_evol_1a.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/orc_ppd_schema_evol_1a.q.out b/ql/src/test/results/clientpositive/orc_ppd_schema_evol_1a.q.out
new file mode 100644
index 0000000..5cefb3f
--- /dev/null
+++ b/ql/src/test/results/clientpositive/orc_ppd_schema_evol_1a.q.out
@@ -0,0 +1,70 @@
+PREHOOK: query: create table unique_1( 
+i int, 
+d double, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@unique_1
+POSTHOOK: query: create table unique_1( 
+i int, 
+d double, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@unique_1
+PREHOOK: query: load data local inpath '../../data/files/unique_1.txt' into table unique_1
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@unique_1
+POSTHOOK: query: load data local inpath '../../data/files/unique_1.txt' into table unique_1
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@unique_1
+PREHOOK: query: create table test1 stored as orc as select * from unique_1
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@unique_1
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test1
+POSTHOOK: query: create table test1 stored as orc as select * from unique_1
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@unique_1
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test1
+POSTHOOK: Lineage: test1.d SIMPLE [(unique_1)unique_1.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: test1.i SIMPLE [(unique_1)unique_1.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: test1.s SIMPLE [(unique_1)unique_1.FieldSchema(name:s, type:string, comment:null), ]
+unique_1.i	unique_1.d	unique_1.s
+PREHOOK: query: alter table test1 change column i i string
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test1
+PREHOOK: Output: default@test1
+PREHOOK: query: select s from test1 where i = '-1591211872'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+#### A masked pattern was here ####
+s
+fred king
+PREHOOK: query: select s from test1 where i = -1591211872
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+#### A masked pattern was here ####
+s
+fred king
+PREHOOK: query: select s from test1 where i = '-1591211872'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+#### A masked pattern was here ####
+s
+fred king
+PREHOOK: query: select s from test1 where i = -1591211872
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+#### A masked pattern was here ####
+s
+fred king

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/ql/src/test/results/clientpositive/orc_ppd_schema_evol_1b.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/orc_ppd_schema_evol_1b.q.out b/ql/src/test/results/clientpositive/orc_ppd_schema_evol_1b.q.out
new file mode 100644
index 0000000..7fa580c
--- /dev/null
+++ b/ql/src/test/results/clientpositive/orc_ppd_schema_evol_1b.q.out
@@ -0,0 +1,124 @@
+PREHOOK: query: create table unique_1( 
+i int, 
+d double, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@unique_1
+POSTHOOK: query: create table unique_1( 
+i int, 
+d double, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@unique_1
+PREHOOK: query: load data local inpath '../../data/files/unique_1.txt' into table unique_1
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@unique_1
+POSTHOOK: query: load data local inpath '../../data/files/unique_1.txt' into table unique_1
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@unique_1
+PREHOOK: query: create table unique_2( 
+i int, 
+d double, 
+s string)
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@unique_2
+POSTHOOK: query: create table unique_2( 
+i int, 
+d double, 
+s string)
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@unique_2
+PREHOOK: query: load data local inpath '../../data/files/unique_2.txt' into table unique_2
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@unique_2
+POSTHOOK: query: load data local inpath '../../data/files/unique_2.txt' into table unique_2
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@unique_2
+PREHOOK: query: create table test_two_files( 
+i int, 
+d double, 
+s string)
+stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_two_files
+POSTHOOK: query: create table test_two_files( 
+i int, 
+d double, 
+s string)
+stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test_two_files
+PREHOOK: query: insert into table test_two_files select * from unique_1 where i <= 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@unique_1
+PREHOOK: Output: default@test_two_files
+POSTHOOK: query: insert into table test_two_files select * from unique_1 where i <= 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@unique_1
+POSTHOOK: Output: default@test_two_files
+POSTHOOK: Lineage: test_two_files.d SIMPLE [(unique_1)unique_1.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: test_two_files.i SIMPLE [(unique_1)unique_1.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: test_two_files.s SIMPLE [(unique_1)unique_1.FieldSchema(name:s, type:string, comment:null), ]
+unique_1.i	unique_1.d	unique_1.s
+PREHOOK: query: insert into table test_two_files select * from unique_2 where i > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@unique_2
+PREHOOK: Output: default@test_two_files
+POSTHOOK: query: insert into table test_two_files select * from unique_2 where i > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@unique_2
+POSTHOOK: Output: default@test_two_files
+POSTHOOK: Lineage: test_two_files.d SIMPLE [(unique_2)unique_2.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: test_two_files.i SIMPLE [(unique_2)unique_2.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: test_two_files.s SIMPLE [(unique_2)unique_2.FieldSchema(name:s, type:string, comment:null), ]
+unique_2.i	unique_2.d	unique_2.s
+PREHOOK: query: alter table test_two_files change column i i string
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test_two_files
+PREHOOK: Output: default@test_two_files
+PREHOOK: query: select s from test_two_files where i = -1591211872
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_two_files
+#### A masked pattern was here ####
+s
+fred king
+PREHOOK: query: select s from test_two_files where i = -1591211872
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_two_files
+#### A masked pattern was here ####
+s
+fred king
+PREHOOK: query: select s from test_two_files where i = '-1591211872'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_two_files
+#### A masked pattern was here ####
+s
+fred king
+PREHOOK: query: select s from test_two_files where i = -1591211872
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_two_files
+#### A masked pattern was here ####
+s
+fred king

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/ql/src/test/results/clientpositive/orc_ppd_schema_evol_2a.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/orc_ppd_schema_evol_2a.q.out b/ql/src/test/results/clientpositive/orc_ppd_schema_evol_2a.q.out
new file mode 100644
index 0000000..5dead1c
--- /dev/null
+++ b/ql/src/test/results/clientpositive/orc_ppd_schema_evol_2a.q.out
@@ -0,0 +1,70 @@
+PREHOOK: query: create table unique_1( 
+i int, 
+d string, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@unique_1
+POSTHOOK: query: create table unique_1( 
+i int, 
+d string, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@unique_1
+PREHOOK: query: load data local inpath '../../data/files/unique_1.txt' into table unique_1
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@unique_1
+POSTHOOK: query: load data local inpath '../../data/files/unique_1.txt' into table unique_1
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@unique_1
+PREHOOK: query: create table test1 stored as orc as select * from unique_1 order by d
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@unique_1
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test1
+POSTHOOK: query: create table test1 stored as orc as select * from unique_1 order by d
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@unique_1
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test1
+POSTHOOK: Lineage: test1.d SIMPLE [(unique_1)unique_1.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: test1.i SIMPLE [(unique_1)unique_1.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: test1.s SIMPLE [(unique_1)unique_1.FieldSchema(name:s, type:string, comment:null), ]
+unique_1.i	unique_1.d	unique_1.s
+PREHOOK: query: alter table test1 change column d d double
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test1
+PREHOOK: Output: default@test1
+PREHOOK: query: select s from test1 where d = -4996703.42
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+#### A masked pattern was here ####
+s
+luke quirinius
+PREHOOK: query: select s from test1 where d = -4996703.42
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+#### A masked pattern was here ####
+s
+luke quirinius
+PREHOOK: query: select s from test1 where d = -4996703.42
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+#### A masked pattern was here ####
+s
+luke quirinius
+PREHOOK: query: select s from test1 where d = -4996703.42
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+#### A masked pattern was here ####
+s
+luke quirinius

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/ql/src/test/results/clientpositive/orc_ppd_schema_evol_2b.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/orc_ppd_schema_evol_2b.q.out b/ql/src/test/results/clientpositive/orc_ppd_schema_evol_2b.q.out
new file mode 100644
index 0000000..b85ec98
--- /dev/null
+++ b/ql/src/test/results/clientpositive/orc_ppd_schema_evol_2b.q.out
@@ -0,0 +1,124 @@
+PREHOOK: query: create table unique_1( 
+i int, 
+d string, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@unique_1
+POSTHOOK: query: create table unique_1( 
+i int, 
+d string, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@unique_1
+PREHOOK: query: load data local inpath '../../data/files/unique_1.txt' into table unique_1
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@unique_1
+POSTHOOK: query: load data local inpath '../../data/files/unique_1.txt' into table unique_1
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@unique_1
+PREHOOK: query: create table unique_2( 
+i int, 
+d string, 
+s string)
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@unique_2
+POSTHOOK: query: create table unique_2( 
+i int, 
+d string, 
+s string)
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@unique_2
+PREHOOK: query: load data local inpath '../../data/files/unique_2.txt' into table unique_2
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@unique_2
+POSTHOOK: query: load data local inpath '../../data/files/unique_2.txt' into table unique_2
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@unique_2
+PREHOOK: query: create table test_two_files( 
+i int, 
+d string, 
+s string)
+stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_two_files
+POSTHOOK: query: create table test_two_files( 
+i int, 
+d string, 
+s string)
+stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test_two_files
+PREHOOK: query: insert into table test_two_files select * from unique_1 where cast(d as double) <= 0 order by cast(d as double)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@unique_1
+PREHOOK: Output: default@test_two_files
+POSTHOOK: query: insert into table test_two_files select * from unique_1 where cast(d as double) <= 0 order by cast(d as double)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@unique_1
+POSTHOOK: Output: default@test_two_files
+POSTHOOK: Lineage: test_two_files.d SIMPLE [(unique_1)unique_1.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: test_two_files.i SIMPLE [(unique_1)unique_1.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: test_two_files.s SIMPLE [(unique_1)unique_1.FieldSchema(name:s, type:string, comment:null), ]
+unique_1.i	unique_1.d	unique_1.s
+PREHOOK: query: insert into table test_two_files select * from unique_2 where cast(d as double) > 0 order by cast(d as double)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@unique_2
+PREHOOK: Output: default@test_two_files
+POSTHOOK: query: insert into table test_two_files select * from unique_2 where cast(d as double) > 0 order by cast(d as double)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@unique_2
+POSTHOOK: Output: default@test_two_files
+POSTHOOK: Lineage: test_two_files.d SIMPLE [(unique_2)unique_2.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: test_two_files.i SIMPLE [(unique_2)unique_2.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: test_two_files.s SIMPLE [(unique_2)unique_2.FieldSchema(name:s, type:string, comment:null), ]
+unique_2.i	unique_2.d	unique_2.s
+PREHOOK: query: alter table test_two_files change column d d double
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test_two_files
+PREHOOK: Output: default@test_two_files
+PREHOOK: query: select s from test_two_files where d = -4996703.42
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_two_files
+#### A masked pattern was here ####
+s
+luke quirinius
+PREHOOK: query: select s from test_two_files where d = -4996703.42
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_two_files
+#### A masked pattern was here ####
+s
+luke quirinius
+PREHOOK: query: select s from test_two_files where d = -4996703.42
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_two_files
+#### A masked pattern was here ####
+s
+luke quirinius
+PREHOOK: query: select s from test_two_files where d = -4996703.42
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_two_files
+#### A masked pattern was here ####
+s
+luke quirinius

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/ql/src/test/results/clientpositive/tez/orc_ppd_schema_evol_1a.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/orc_ppd_schema_evol_1a.q.out b/ql/src/test/results/clientpositive/tez/orc_ppd_schema_evol_1a.q.out
new file mode 100644
index 0000000..5cefb3f
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/orc_ppd_schema_evol_1a.q.out
@@ -0,0 +1,70 @@
+PREHOOK: query: create table unique_1( 
+i int, 
+d double, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@unique_1
+POSTHOOK: query: create table unique_1( 
+i int, 
+d double, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@unique_1
+PREHOOK: query: load data local inpath '../../data/files/unique_1.txt' into table unique_1
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@unique_1
+POSTHOOK: query: load data local inpath '../../data/files/unique_1.txt' into table unique_1
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@unique_1
+PREHOOK: query: create table test1 stored as orc as select * from unique_1
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@unique_1
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test1
+POSTHOOK: query: create table test1 stored as orc as select * from unique_1
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@unique_1
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test1
+POSTHOOK: Lineage: test1.d SIMPLE [(unique_1)unique_1.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: test1.i SIMPLE [(unique_1)unique_1.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: test1.s SIMPLE [(unique_1)unique_1.FieldSchema(name:s, type:string, comment:null), ]
+unique_1.i	unique_1.d	unique_1.s
+PREHOOK: query: alter table test1 change column i i string
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test1
+PREHOOK: Output: default@test1
+PREHOOK: query: select s from test1 where i = '-1591211872'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+#### A masked pattern was here ####
+s
+fred king
+PREHOOK: query: select s from test1 where i = -1591211872
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+#### A masked pattern was here ####
+s
+fred king
+PREHOOK: query: select s from test1 where i = '-1591211872'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+#### A masked pattern was here ####
+s
+fred king
+PREHOOK: query: select s from test1 where i = -1591211872
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+#### A masked pattern was here ####
+s
+fred king

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/ql/src/test/results/clientpositive/tez/orc_ppd_schema_evol_1b.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/orc_ppd_schema_evol_1b.q.out b/ql/src/test/results/clientpositive/tez/orc_ppd_schema_evol_1b.q.out
new file mode 100644
index 0000000..7fa580c
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/orc_ppd_schema_evol_1b.q.out
@@ -0,0 +1,124 @@
+PREHOOK: query: create table unique_1( 
+i int, 
+d double, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@unique_1
+POSTHOOK: query: create table unique_1( 
+i int, 
+d double, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@unique_1
+PREHOOK: query: load data local inpath '../../data/files/unique_1.txt' into table unique_1
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@unique_1
+POSTHOOK: query: load data local inpath '../../data/files/unique_1.txt' into table unique_1
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@unique_1
+PREHOOK: query: create table unique_2( 
+i int, 
+d double, 
+s string)
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@unique_2
+POSTHOOK: query: create table unique_2( 
+i int, 
+d double, 
+s string)
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@unique_2
+PREHOOK: query: load data local inpath '../../data/files/unique_2.txt' into table unique_2
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@unique_2
+POSTHOOK: query: load data local inpath '../../data/files/unique_2.txt' into table unique_2
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@unique_2
+PREHOOK: query: create table test_two_files( 
+i int, 
+d double, 
+s string)
+stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_two_files
+POSTHOOK: query: create table test_two_files( 
+i int, 
+d double, 
+s string)
+stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test_two_files
+PREHOOK: query: insert into table test_two_files select * from unique_1 where i <= 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@unique_1
+PREHOOK: Output: default@test_two_files
+POSTHOOK: query: insert into table test_two_files select * from unique_1 where i <= 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@unique_1
+POSTHOOK: Output: default@test_two_files
+POSTHOOK: Lineage: test_two_files.d SIMPLE [(unique_1)unique_1.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: test_two_files.i SIMPLE [(unique_1)unique_1.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: test_two_files.s SIMPLE [(unique_1)unique_1.FieldSchema(name:s, type:string, comment:null), ]
+unique_1.i	unique_1.d	unique_1.s
+PREHOOK: query: insert into table test_two_files select * from unique_2 where i > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@unique_2
+PREHOOK: Output: default@test_two_files
+POSTHOOK: query: insert into table test_two_files select * from unique_2 where i > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@unique_2
+POSTHOOK: Output: default@test_two_files
+POSTHOOK: Lineage: test_two_files.d SIMPLE [(unique_2)unique_2.FieldSchema(name:d, type:double, comment:null), ]
+POSTHOOK: Lineage: test_two_files.i SIMPLE [(unique_2)unique_2.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: test_two_files.s SIMPLE [(unique_2)unique_2.FieldSchema(name:s, type:string, comment:null), ]
+unique_2.i	unique_2.d	unique_2.s
+PREHOOK: query: alter table test_two_files change column i i string
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test_two_files
+PREHOOK: Output: default@test_two_files
+PREHOOK: query: select s from test_two_files where i = -1591211872
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_two_files
+#### A masked pattern was here ####
+s
+fred king
+PREHOOK: query: select s from test_two_files where i = -1591211872
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_two_files
+#### A masked pattern was here ####
+s
+fred king
+PREHOOK: query: select s from test_two_files where i = '-1591211872'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_two_files
+#### A masked pattern was here ####
+s
+fred king
+PREHOOK: query: select s from test_two_files where i = -1591211872
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_two_files
+#### A masked pattern was here ####
+s
+fred king

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/ql/src/test/results/clientpositive/tez/orc_ppd_schema_evol_2a.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/orc_ppd_schema_evol_2a.q.out b/ql/src/test/results/clientpositive/tez/orc_ppd_schema_evol_2a.q.out
new file mode 100644
index 0000000..5dead1c
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/orc_ppd_schema_evol_2a.q.out
@@ -0,0 +1,70 @@
+PREHOOK: query: create table unique_1( 
+i int, 
+d string, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@unique_1
+POSTHOOK: query: create table unique_1( 
+i int, 
+d string, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@unique_1
+PREHOOK: query: load data local inpath '../../data/files/unique_1.txt' into table unique_1
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@unique_1
+POSTHOOK: query: load data local inpath '../../data/files/unique_1.txt' into table unique_1
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@unique_1
+PREHOOK: query: create table test1 stored as orc as select * from unique_1 order by d
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@unique_1
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test1
+POSTHOOK: query: create table test1 stored as orc as select * from unique_1 order by d
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@unique_1
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test1
+POSTHOOK: Lineage: test1.d SIMPLE [(unique_1)unique_1.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: test1.i SIMPLE [(unique_1)unique_1.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: test1.s SIMPLE [(unique_1)unique_1.FieldSchema(name:s, type:string, comment:null), ]
+unique_1.i	unique_1.d	unique_1.s
+PREHOOK: query: alter table test1 change column d d double
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test1
+PREHOOK: Output: default@test1
+PREHOOK: query: select s from test1 where d = -4996703.42
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+#### A masked pattern was here ####
+s
+luke quirinius
+PREHOOK: query: select s from test1 where d = -4996703.42
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+#### A masked pattern was here ####
+s
+luke quirinius
+PREHOOK: query: select s from test1 where d = -4996703.42
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+#### A masked pattern was here ####
+s
+luke quirinius
+PREHOOK: query: select s from test1 where d = -4996703.42
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test1
+#### A masked pattern was here ####
+s
+luke quirinius

http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/ql/src/test/results/clientpositive/tez/orc_ppd_schema_evol_2b.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/orc_ppd_schema_evol_2b.q.out b/ql/src/test/results/clientpositive/tez/orc_ppd_schema_evol_2b.q.out
new file mode 100644
index 0000000..b85ec98
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/orc_ppd_schema_evol_2b.q.out
@@ -0,0 +1,124 @@
+PREHOOK: query: create table unique_1( 
+i int, 
+d string, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@unique_1
+POSTHOOK: query: create table unique_1( 
+i int, 
+d string, 
+s string) 
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@unique_1
+PREHOOK: query: load data local inpath '../../data/files/unique_1.txt' into table unique_1
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@unique_1
+POSTHOOK: query: load data local inpath '../../data/files/unique_1.txt' into table unique_1
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@unique_1
+PREHOOK: query: create table unique_2( 
+i int, 
+d string, 
+s string)
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@unique_2
+POSTHOOK: query: create table unique_2( 
+i int, 
+d string, 
+s string)
+row format delimited 
+fields terminated by '|' 
+stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@unique_2
+PREHOOK: query: load data local inpath '../../data/files/unique_2.txt' into table unique_2
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@unique_2
+POSTHOOK: query: load data local inpath '../../data/files/unique_2.txt' into table unique_2
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@unique_2
+PREHOOK: query: create table test_two_files( 
+i int, 
+d string, 
+s string)
+stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_two_files
+POSTHOOK: query: create table test_two_files( 
+i int, 
+d string, 
+s string)
+stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test_two_files
+PREHOOK: query: insert into table test_two_files select * from unique_1 where cast(d as double) <= 0 order by cast(d as double)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@unique_1
+PREHOOK: Output: default@test_two_files
+POSTHOOK: query: insert into table test_two_files select * from unique_1 where cast(d as double) <= 0 order by cast(d as double)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@unique_1
+POSTHOOK: Output: default@test_two_files
+POSTHOOK: Lineage: test_two_files.d SIMPLE [(unique_1)unique_1.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: test_two_files.i SIMPLE [(unique_1)unique_1.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: test_two_files.s SIMPLE [(unique_1)unique_1.FieldSchema(name:s, type:string, comment:null), ]
+unique_1.i	unique_1.d	unique_1.s
+PREHOOK: query: insert into table test_two_files select * from unique_2 where cast(d as double) > 0 order by cast(d as double)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@unique_2
+PREHOOK: Output: default@test_two_files
+POSTHOOK: query: insert into table test_two_files select * from unique_2 where cast(d as double) > 0 order by cast(d as double)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@unique_2
+POSTHOOK: Output: default@test_two_files
+POSTHOOK: Lineage: test_two_files.d SIMPLE [(unique_2)unique_2.FieldSchema(name:d, type:string, comment:null), ]
+POSTHOOK: Lineage: test_two_files.i SIMPLE [(unique_2)unique_2.FieldSchema(name:i, type:int, comment:null), ]
+POSTHOOK: Lineage: test_two_files.s SIMPLE [(unique_2)unique_2.FieldSchema(name:s, type:string, comment:null), ]
+unique_2.i	unique_2.d	unique_2.s
+PREHOOK: query: alter table test_two_files change column d d double
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test_two_files
+PREHOOK: Output: default@test_two_files
+PREHOOK: query: select s from test_two_files where d = -4996703.42
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_two_files
+#### A masked pattern was here ####
+s
+luke quirinius
+PREHOOK: query: select s from test_two_files where d = -4996703.42
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_two_files
+#### A masked pattern was here ####
+s
+luke quirinius
+PREHOOK: query: select s from test_two_files where d = -4996703.42
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_two_files
+#### A masked pattern was here ####
+s
+luke quirinius
+PREHOOK: query: select s from test_two_files where d = -4996703.42
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_two_files
+#### A masked pattern was here ####
+s
+luke quirinius


[3/4] hive git commit: HIVE-14214: ORC Schema Evolution and Predicate Push Down do not work together (no rows returned) (Matt McCline, reviewed by Prasanth Jayachandran/Owen O'Malley)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/data/files/unique_1.txt
----------------------------------------------------------------------
diff --git a/data/files/unique_1.txt b/data/files/unique_1.txt
new file mode 100644
index 0000000..2f56798
--- /dev/null
+++ b/data/files/unique_1.txt
@@ -0,0 +1,5000 @@
+-1591211872|4830897.73|fred king
+1500342260|-3488786.37|xavier laertes
+2077949809|-2246321.41|holly nixon
+-907169128|-305379.39|bob davidson
+-1956877857|-4582042.21|priscilla nixon
+-126017579|-2303832.42|luke ichabod
+-949589797|-1689997.65|zach king
+43664721|2397175.04|gabriella van buren
+352713100|-3637886.14|zach laertes
+785849488|1029760.51|priscilla zipper
+-1187718083|-4688740.11|wendy steinbeck
+638855099|3334150.19|yuri robinson
+1360512016|4890112.98|mike falkner
+-560766951|-3436616.13|gabriella zipper
+-997311370|2484949.03|calvin zipper
+-1750806361|3433311.01|irene xylophone
+1771266072|2610845.98|sarah young
+1206662086|-557900.68|sarah young
+1018161833|-2839778.88|luke ellison
+-233450233|1561574.7|rachel young
+2135730664|427702.98|yuri van buren
+-742004641|-3344922.34|luke quirinius
+-1495225427|-125201.2|quinn quirinius
+1360432779|-1470131.58|bob zipper
+725159392|2585645.79|xavier van buren
+-108959161|-4928399.04|david thompson
+2099567278|270287.58|david johnson
+2036272259|-3308980.61|rachel ovid
+84487758|-4440484.66|bob allen
+1729832454|-3651942.29|katie zipper
+-1414539986|-3101612.08|nick brown
+-1409520913|-889861.38|fred garcia
+614658165|1638078.92|jessica miller
+433322241|-3263380.42|ethan van buren
+-1836790760|-3460016.21|ulysses zipper
+514124770|3332987.47|ethan xylophone
+-1741341474|1195933.74|ulysses davidson
+1339697781|-2877988.38|ethan ellison
+1484416573|-242409.11|holly white
+1644195617|-2172371.16|priscilla carson
+1582169820|2719884.4|david falkner
+-857954127|2057802.59|quinn quirinius
+1111613440|2965834.06|sarah van buren
+163324919|4823964.16|holly ovid
+-2049560730|-1041182.08|gabriella carson
+-194836844|4185527.1|yuri johnson
+-798256073|1214878.55|sarah thompson
+1732677977|88997.5|rachel falkner
+-1671497967|-768822.51|nick ovid
+-1720650419|-564965.37|gabriella white
+401087068|-867794.41|quinn ovid
+695423035|-2813948.16|priscilla quirinius
+-788647062|-2569600.48|irene underhill
+-341559728|-2222179.75|holly carson
+-92903933|2646688.23|nick king
+-414968782|2975642.54|mike carson
+-162010243|-2061605.22|oscar hernandez
+-1293925994|-2789444.17|gabriella hernandez
+621581170|-92768.99|calvin brown
+-1930502583|-5006.57|oscar brown
+1956478881|-4843662.2|quinn zipper
+-1543824359|4976187.61|gabriella nixon
+-2100146780|3481675.87|oscar steinbeck
+-164661721|-4658286.17|luke king
+-343760763|-799554.15|david white
+1325079181|-1103876.84|tom carson
+-923207330|-4150611.95|alice quirinius
+-1169914527|693143.87|yuri ellison
+-1347724936|341595.02|calvin garcia
+1116572771|3919277.55|tom polk
+-203484755|-1654189.34|wendy steinbeck
+-353963749|2993273.72|katie ellison
+-165299716|-2311198.57|david xylophone
+78809852|-359088.38|ethan ichabod
+-1111924341|-1864045.42|katie davidson
+-1034662426|4479653.08|mike ovid
+-1369642355|-4531193.31|sarah laertes
+1852467498|-4626145.99|oscar brown
+-340392248|-4863336.94|fred davidson
+-1955886635|-3582217.06|holly thompson
+383432876|-77986.23|jessica brown
+-1950515475|1353180.11|wendy falkner
+-1653446083|-1181302.01|fred brown
+1679632607|-4056673.46|fred carson
+-1503664589|-4452662.45|gabriella carson
+318721385|527804.38|victor white
+-722560719|4837128.07|rachel quirinius
+1682878333|-4464957.68|priscilla brown
+547014612|-4742569.73|luke allen
+-906873043|2062555.11|luke white
+539339785|893798.21|oscar white
+-1676994691|1074333.01|david garcia
+-1756430605|1367230.01|tom brown
+900810912|-1215279.24|priscilla carson
+-2107890121|-3845783.96|gabriella xylophone
+-911185117|-3619826.29|luke polk
+-1252500827|4492627.5|xavier white
+-415118128|-778886.81|victor quirinius
+2001369948|-3957855.19|alice robinson
+2113725065|-800868.06|calvin underhill
+1406544901|3985767.04|irene king
+-1029264809|-4834537.33|ulysses white
+-1630853696|-4968566.09|alice johnson
+514596663|-1493823.14|bob king
+-499410494|4926357.78|priscilla davidson
+629392556|-3309728.22|sarah ovid
+1776351730|952788.62|jessica ellison
+855597564|3728696.25|rachel hernandez
+1128054493|3715387.4|ethan carson
+-1257251436|3689686.75|holly davidson
+184787906|733670.5|ulysses thompson
+374462224|-791948.14|tom thompson
+567779525|2613566.67|irene ellison
+1971303133|1146825.47|quinn falkner
+686127199|-4113530.41|luke van buren
+1711035500|-1606690.76|wendy polk
+1850427547|-3889793.43|tom polk
+930960483|815407.61|quinn brown
+93784525|-3458609.92|fred young
+-1349900696|1422430.1|fred laertes
+971934654|1621304.94|luke falkner
+920841795|-4434706.55|mike garcia
+1095378825|1567782.51|calvin davidson
+1423623829|1857917.66|ulysses davidson
+-440184935|-1135587.53|xavier underhill
+1225818968|1657231.82|holly miller
+1363097727|4116328.95|luke allen
+-1283286820|-3653890.26|rachel van buren
+-241544019|-3647875.47|priscilla quirinius
+1024480254|-4704928.33|wendy robinson
+-1780671957|2613683.68|xavier johnson
+500107990|-1729774.07|calvin underhill
+1675547871|-2860679.38|bob steinbeck
+479587591|1283985.33|rachel laertes
+2144661772|-3686219.42|irene underhill
+2088808814|-556155.22|yuri thompson
+-68124409|-3551186.68|gabriella quirinius
+385358584|-3950288.37|tom hernandez
+-390031790|-2487318.65|gabriella underhill
+-1160232778|4389419.69|mike zipper
+898476463|626889.06|xavier miller
+-959038072|3914032.54|nick ichabod
+287542974|3616945.33|fred hernandez
+-1053197819|-419810.01|oscar miller
+-1435066572|4429089.37|yuri allen
+-1268402342|2861975.49|bob carson
+828094176|-3336805.84|victor robinson
+1873374296|-4523389.99|tom johnson
+-77163267|3636008.8|nick ovid
+-374573576|4281663.64|alice johnson
+908071025|4456711.44|rachel thompson
+252420841|-4487842.88|gabriella white
+167613459|4021140.98|luke brown
+-680407533|4732145.79|xavier johnson
+-924230387|1161201.88|nick brown
+947658251|2098792.72|quinn hernandez
+-1383970578|-346098.33|nick ovid
+-642832392|-2159495.3|calvin thompson
+-1035374058|-544375.35|bob laertes
+-2084949430|1918890.74|luke nixon
+-205932425|2180664.21|irene garcia
+1292253268|1308395.64|gabriella quirinius
+1561031138|-4103546.28|yuri ovid
+176859583|-1593679.65|sarah steinbeck
+-2135545762|4188472.3|tom hernandez
+1148895532|-200016.76|mike allen
+641226285|4097694.42|bob carson
+897958839|-1478474.47|sarah allen
+1538115233|-32684.47|ethan hernandez
+-1415440615|-4513146.01|holly falkner
+-1794607169|1073139.82|ulysses robinson
+1898966663|-710412.54|david miller
+-689644451|4238477.45|ethan polk
+-1645549888|-2741539.85|jessica miller
+933494612|3476075.18|gabriella underhill
+-1899866198|660839.64|victor falkner
+-1128359985|-3156737.39|fred van buren
+1405875863|-694671.99|mike johnson
+1805216724|-3484927.51|jessica steinbeck
+1156230683|-896167.0|rachel robinson
+1197128109|-533109.21|priscilla ovid
+-1831804707|3901716.93|ulysses polk
+207788469|-3391447.94|zach brown
+-1337478407|-2047728.02|tom thompson
+-234507860|-471218.04|david white
+-1579154990|-71526.56|rachel xylophone
+1819146154|3047064.09|yuri xylophone
+-473795496|-1628333.08|holly xylophone
+124515911|-931142.69|holly allen
+1878332981|2878065.28|rachel zipper
+1193989264|-3663849.34|xavier nixon
+-1269150108|-672675.52|gabriella nixon
+339256280|309617.9|ulysses carson
+-1389869559|-2116468.61|wendy robinson
+1238168217|4863329.2|sarah garcia
+1959530171|-2968869.29|quinn thompson
+-1899319593|-570852.89|yuri steinbeck
+-730753152|541424.34|xavier king
+-434320453|4033081.49|oscar white
+1776997899|-2102676.48|katie davidson
+289092971|638082.66|victor robinson
+-1804794299|1004578.36|tom davidson
+-1678298285|-3899620.61|zach ellison
+1523291115|3392805.46|ulysses garcia
+1045085242|955033.61|wendy king
+649313009|1771288.86|rachel carson
+-329760484|-3743379.93|irene brown
+-699778825|-2360032.42|alice van buren
+-1422761361|4004643.4|priscilla zipper
+879435471|4779564.46|irene ovid
+288534869|3826019.39|oscar steinbeck
+204602158|-2997316.64|bob nixon
+319379717|-2050962.62|katie ichabod
+464701386|-3466410.28|yuri van buren
+556148272|3053414.46|zach quirinius
+2041596047|-4088958.35|calvin xylophone
+1538140509|-2620992.8|ulysses xylophone
+954103992|2968242.81|katie carson
+-492571014|-1662305.7|fred zipper
+284443633|1078222.42|alice king
+187602687|3051991.96|fred thompson
+808038858|-2760973.41|oscar ovid
+-1736739447|4914102.42|bob van buren
+-114104285|1223131.81|rachel xylophone
+1052780715|3029481.08|wendy brown
+-988902061|1470421.27|victor johnson
+123461737|32577.86|calvin van buren
+-386257596|-3906793.34|alice polk
+-725703515|-2359596.13|ulysses van buren
+1004659894|-2714469.87|holly ichabod
+-908370870|-2668988.7|zach hernandez
+1863044194|4610126.69|gabriella ellison
+-1458238403|2226649.31|victor ellison
+1963653556|3472795.82|zach ovid
+1547678243|168432.59|ulysses king
+404913496|432737.76|irene miller
+-1439636829|1603094.66|priscilla thompson
+1197285335|2939089.91|ethan hernandez
+-1545686100|-1435895.89|yuri garcia
+944425700|-4789254.6|xavier brown
+1667681002|-777823.72|priscilla ovid
+889594424|4888952.93|katie underhill
+-860785285|578413.2|yuri zipper
+-2028283639|-3177113.32|david polk
+-473773730|150628.31|jessica ovid
+-2073271942|3412261.73|ethan carson
+1273416314|-2441773.96|gabriella laertes
+-1521305179|-3543715.02|fred young
+-1260538876|-2034191.4|oscar thompson
+1618732543|-3969043.51|luke falkner
+1964944531|4834625.24|jessica thompson
+-762762415|4460237.38|mike ellison
+-1078157714|1727043.18|luke zipper
+-1165893009|-2357753.42|jessica thompson
+314670742|4980096.25|fred falkner
+2025774717|-1562766.72|luke zipper
+1832275143|-4716584.21|xavier ovid
+-955096469|-1687737.06|ethan underhill
+-1109699734|474391.64|alice falkner
+71605454|-3575242.31|ulysses underhill
+924711453|-303077.01|priscilla king
+-1614012020|850511.51|oscar nixon
+-1740331403|1056380.37|wendy carson
+-1634799948|4650346.27|oscar king
+-20840979|2686174.51|xavier robinson
+-1721579871|3580536.4|luke davidson
+-1980812980|-1290179.49|rachel garcia
+-1935926456|55135.39|jessica nixon
+1600658668|1271857.3|priscilla zipper
+2020081448|-516499.5|luke zipper
+556190365|-705430.33|mike miller
+-71852436|-2607757.64|victor steinbeck
+1872269596|4615767.03|gabriella ichabod
+-602392819|-1584736.37|sarah ichabod
+-893683108|4535407.83|mike white
+602435875|4498920.89|calvin allen
+-579070706|118857.25|katie falkner
+-315103069|-410937.88|victor young
+-1013240019|-4743716.09|quinn hernandez
+-1535786960|-4432481.3|bob xylophone
+2022735475|3752730.0|holly carson
+-736968481|-3601103.05|holly underhill
+379502863|-3669950.18|wendy thompson
+-2031620882|4620521.03|holly brown
+1220719593|4259308.27|david ellison
+1546523610|1075798.3|holly garcia
+1215769937|4328004.79|sarah steinbeck
+771880045|1618848.24|jessica falkner
+-1881929211|4024855.6|katie davidson
+-360948343|-2018418.8|rachel thompson
+1174222770|-81019.99|sarah thompson
+-1029596970|3858291.33|alice allen
+286544794|-1779443.23|ulysses king
+-333985098|-1367382.9|holly hernandez
+1877400787|1897138.26|jessica brown
+780331255|663469.08|fred allen
+-601280172|-850187.62|irene davidson
+-872544703|108349.04|alice nixon
+-1756151325|-2382213.32|ethan xylophone
+-20053456|467677.63|jessica white
+-2062128015|-2056756.31|tom xylophone
+1849149083|-646649.79|sarah nixon
+-1309209208|3491694.14|tom steinbeck
+1415325826|4734930.83|david ovid
+625109210|-1069081.1|ethan falkner
+42129282|-3736003.03|irene young
+1882740241|-1975192.8|wendy steinbeck
+1745064180|1870560.76|ulysses davidson
+-100337042|-1323986.69|xavier king
+-1875010564|-2683625.81|alice polk
+-1803906851|1279762.27|mike allen
+-868779478|-1719084.14|xavier young
+-995219065|-2902730.97|yuri white
+-46709618|2014682.67|ethan white
+2121805574|-2740163.7|jessica laertes
+86273855|551661.18|nick nixon
+-786582836|5241.37|quinn davidson
+292042180|-3488247.38|fred white
+1225839442|-3149645.07|luke hernandez
+-211365654|-2242593.51|priscilla brown
+1648053817|3239173.14|calvin zipper
+1722964708|1486612.2|sarah davidson
+-1045934756|-3033774.33|david brown
+-439615280|2101063.79|ethan carson
+706939251|-1729057.83|sarah young
+-1780391700|-4411505.87|luke ellison
+2001978876|-1589923.8|yuri laertes
+-1837315290|-1713577.8|fred carson
+-437763767|-2662854.82|oscar davidson
+-847833650|-3478986.07|tom ellison
+-488650497|-1299906.55|alice ellison
+834267786|1597625.88|katie white
+2108491397|4911029.88|priscilla johnson
+1403021290|912099.99|priscilla johnson
+-529328123|3430469.07|nick laertes
+305775138|3909004.07|quinn xylophone
+-2135754242|-1482568.63|mike ellison
+-12860972|2480924.07|zach white
+897020595|-1675497.64|wendy quirinius
+863545845|-1438765.22|zach falkner
+490597333|-245374.88|ulysses quirinius
+744603197|-1622347.58|david steinbeck
+-356335578|-4631217.86|fred allen
+1261136266|4967076.89|fred hernandez
+-52442603|-10658.49|yuri nixon
+195391557|2859136.82|xavier ellison
+-923253360|-2291894.12|ethan carson
+-2101494592|3172584.36|katie xylophone
+-1303336341|951113.16|david allen
+-662586394|-683726.57|ulysses young
+486725168|-4639711.69|holly robinson
+-1231225109|-3412727.61|david ellison
+-1383269568|-370589.98|wendy young
+-1352303945|-1051999.72|fred white
+429795452|4396902.82|holly laertes
+-180266954|-2632656.82|david laertes
+1222060031|-1924823.89|xavier robinson
+-1899468568|1662861.72|priscilla robinson
+577158281|-3256821.84|yuri ellison
+-441054921|-3883249.72|yuri brown
+1420377252|374678.96|jessica underhill
+-320399746|880567.49|alice falkner
+706019257|2984380.93|katie polk
+-150228786|3757289.9|xavier white
+1021203588|1989036.72|priscilla ichabod
+359946444|4320593.62|quinn ichabod
+430021033|1927441.89|jessica ellison
+971352130|4675506.69|victor carson
+765648988|-3527838.66|irene van buren
+-738527178|2338939.8|mike steinbeck
+643173977|4649470.9|fred ichabod
+1568002462|3114194.6|oscar allen
+-2041704547|4271667.35|irene ellison
+1628608032|362752.06|katie young
+-1300356707|-2542956.21|gabriella white
+-403147702|-1258386.92|yuri hernandez
+-93984523|703808.25|nick young
+-298907843|3149468.37|victor zipper
+483108802|3763372.87|ethan steinbeck
+-1302592539|-2627071.72|alice falkner
+1492835871|440733.01|katie hernandez
+-705585674|3355025.41|priscilla miller
+-154685564|4140084.22|priscilla quirinius
+1433857384|1875281.48|fred garcia
+2004805358|3966069.36|rachel johnson
+1908416822|1845807.21|tom nixon
+-534695634|2585457.98|mike king
+1463797375|-231514.79|irene nixon
+-494964839|4796553.27|irene laertes
+-381850783|661496.76|victor quirinius
+-1937068270|4834534.36|mike quirinius
+-567641310|4314605.23|holly king
+-75979200|2030401.63|irene steinbeck
+1096019337|4079639.67|zach quirinius
+-1050315988|-3124911.7|victor brown
+-1168681613|-3182693.35|sarah davidson
+445903813|508383.28|gabriella polk
+1058181881|-1884878.29|victor young
+-923642351|2877520.11|david ellison
+99023835|-759722.56|rachel steinbeck
+1019915217|4945846.87|katie davidson
+-622280478|2495228.26|victor robinson
+-2023408867|-3259660.98|priscilla falkner
+701022566|-1613503.01|zach polk
+500776988|-3458170.23|ethan laertes
+2065067613|1888571.22|victor brown
+206155227|-765978.52|luke laertes
+150398047|-991457.45|quinn quirinius
+1223590656|2367064.41|katie miller
+-1398173523|-642234.23|yuri young
+-714847048|-1773752.68|gabriella miller
+-1685154008|2065226.98|zach davidson
+1716372802|3192233.67|xavier ellison
+-916301906|-4206105.83|mike miller
+-1729762076|-3783754.94|ethan van buren
+-1547178149|2591089.6|wendy davidson
+-1636705505|3100970.75|irene underhill
+-762055011|4836833.96|mike young
+412787591|2014353.15|nick underhill
+46405412|-3539178.69|ulysses white
+1742886036|342366.4|bob quirinius
+-1378880241|-4430467.75|quinn thompson
+1744838701|87817.43|luke johnson
+-927572768|-3162451.48|rachel brown
+454784241|-637596.4|sarah thompson
+69092394|1395103.95|katie ichabod
+287048810|-3838062.25|wendy carson
+-2057896718|4887731.6|rachel ellison
+-1562069599|-2896197.23|ethan young
+-403401686|-4019201.93|yuri ichabod
+1757684944|4017338.04|wendy allen
+-787168795|-1009295.37|ethan ovid
+108585010|-4386006.56|victor white
+1967891983|-1781259.13|mike johnson
+537911059|53506.03|priscilla johnson
+391960262|-3799201.75|wendy steinbeck
+557190059|-1770151.21|xavier young
+844731818|1765005.46|jessica van buren
+387641721|-3316279.19|ulysses robinson
+-2066847804|-3863501.21|xavier nixon
+-261420785|-1562308.6|nick miller
+-1562617908|-950196.9|gabriella robinson
+-1302364280|-3331299.86|quinn steinbeck
+-1417024255|3244908.32|fred ichabod
+-1904014548|1603918.42|jessica davidson
+1405085891|-404980.99|david davidson
+1895182313|361144.33|katie ovid
+440548281|-1039665.51|yuri underhill
+1653052839|-2354832.27|quinn carson
+1837623073|2142255.0|holly garcia
+1681952673|-110028.31|david van buren
+1250408861|1551359.37|rachel thompson
+-1359741107|2210769.06|tom garcia
+1635554985|-4684931.89|alice brown
+35858036|-4475679.06|sarah king
+238308968|2308165.89|alice robinson
+-2107305069|-218439.15|luke hernandez
+-1938847849|2362039.69|fred laertes
+-683546479|2879980.84|victor falkner
+-47648830|1506942.55|ethan robinson
+-485843090|1289020.96|priscilla white
+842817044|-1279751.21|sarah van buren
+132837761|-3084814.06|jessica allen
+1777591126|3615294.02|priscilla hernandez
+-1171786407|3311822.65|ethan johnson
+993424888|2687986.86|alice underhill
+-1446988713|2552831.33|gabriella robinson
+-2017914057|-3745038.85|wendy king
+-1432111965|4841901.48|alice quirinius
+-658743614|-2893785.08|nick polk
+-1895428848|-1400001.43|luke van buren
+-1139535761|2267751.61|xavier king
+1273389047|101171.2|sarah allen
+476998611|-4455841.28|luke laertes
+-1947476150|3844890.77|luke hernandez
+2128977894|-4941895.91|quinn falkner
+-1269657805|3349170.75|jessica zipper
+-1908026126|-1291861.86|oscar ichabod
+-408342995|2821237.01|sarah robinson
+-1370165891|-1599390.02|wendy laertes
+1538836577|310927.76|yuri polk
+-1790565636|3505565.09|ethan van buren
+331078036|-4139913.54|quinn xylophone
+1090437473|-2238703.1|mike steinbeck
+-108245991|4376700.98|quinn white
+-1124408086|-2021905.46|luke allen
+-592352129|-588439.55|gabriella nixon
+-1046785100|-1485505.4|david allen
+2072862969|-2471719.44|wendy ichabod
+1806166279|3968007.02|irene van buren
+-1367037771|21207.4|calvin johnson
+-661808548|338355.91|fred carson
+-1286018427|1821197.1|oscar robinson
+-633178509|-3582386.37|zach allen
+-1003111491|-812498.22|alice steinbeck
+1326787881|-2393950.89|rachel carson
+454317851|-2166716.55|tom zipper
+741753554|3359136.31|wendy ovid
+-469219073|-1897116.01|priscilla allen
+-1082254290|4289474.37|jessica polk
+-1769983654|-243937.87|irene quirinius
+2044797016|-4612796.16|rachel ichabod
+-1056721612|-3921554.96|priscilla zipper
+2078823511|3260040.66|wendy johnson
+1834672275|578663.21|sarah falkner
+-774594565|3391646.42|katie ichabod
+204431125|-4546442.7|bob falkner
+-463835566|3877272.54|calvin zipper
+-1050123089|-4739573.49|oscar xylophone
+1921852398|957246.38|yuri young
+1707979076|-1228167.43|alice van buren
+1393165212|-3659747.48|rachel ovid
+1558911267|-82225.28|jessica thompson
+1420435043|4720838.51|gabriella zipper
+-1647429257|1832890.7|rachel garcia
+894821865|4847346.73|jessica thompson
+421317455|3216875.81|mike nixon
+353073286|-4555272.13|ulysses young
+1544564963|3368437.15|rachel carson
+-1222847302|-4456736.81|rachel zipper
+109490176|4563309.2|oscar thompson
+718036404|2167634.3|wendy laertes
+1511793135|-902283.07|nick quirinius
+1327861047|-3625044.48|rachel underhill
+2117207100|-3145572.68|jessica polk
+1461970683|-4202140.0|priscilla ovid
+-1175152871|3941925.37|katie polk
+1416843348|-856335.9|fred van buren
+-372087468|-1105874.09|holly underhill
+1627415254|404435.67|victor nixon
+-842858718|1463754.69|gabriella miller
+-1075561692|4512684.41|sarah johnson
+1165896743|791821.32|rachel johnson
+1811559721|-4554287.3|katie steinbeck
+1404514944|-2015171.2|zach ovid
+697634167|-3238778.58|priscilla carson
+-1903667269|1349843.64|ethan miller
+-968914053|-324940.92|mike underhill
+-307411451|251969.08|xavier falkner
+-1736426820|384633.43|ethan xylophone
+-1143287047|-1497791.4|sarah allen
+-1818599054|3622565.35|holly laertes
+-1906927925|-1501569.62|calvin garcia
+-1562091247|-3080059.63|katie brown
+-1006165561|-3937406.1|jessica quirinius
+-1326542445|-2268699.1|katie johnson
+-84395166|897591.18|quinn laertes
+-915373395|-219124.84|xavier brown
+795263307|3473831.39|xavier johnson
+-205132231|4506758.87|luke zipper
+177358396|3946963.88|ulysses quirinius
+-1539138190|2545943.72|david robinson
+-531380354|872603.17|tom thompson
+687120085|-3282482.02|sarah brown
+1454409048|-4443547.56|katie hernandez
+28277964|2651633.96|ulysses thompson
+66948590|2882310.57|victor steinbeck
+206585191|-2218590.56|nick van buren
+-394351826|1446915.13|holly allen
+835772879|-3634470.78|luke falkner
+2061792216|-740962.62|quinn underhill
+-1105170370|2220935.8|luke zipper
+-2093019941|-112569.73|quinn quirinius
+2033802642|1283840.06|luke nixon
+-1143274649|4318667.42|holly ellison
+-778568724|2429422.79|jessica nixon
+1697931659|-2599895.91|oscar white
+-1221887545|1416723.72|priscilla nixon
+248120900|-4710192.68|wendy steinbeck
+109676938|-3870775.2|fred nixon
+1051471661|-2478558.11|yuri carson
+982256537|4663415.61|alice carson
+-1734322839|33224.13|sarah falkner
+-1477731019|-3282173.77|sarah hernandez
+761319509|3304617.67|jessica white
+851299537|-1422894.82|holly ichabod
+-1223790557|-546200.56|ulysses brown
+-1394902849|-4701786.55|katie allen
+1713323885|346564.06|fred polk
+551982883|3684852.69|quinn robinson
+-1942935965|-740957.37|quinn thompson
+1659299512|-593486.37|fred king
+-255978719|2690553.72|jessica johnson
+-921903659|3137616.52|jessica hernandez
+-681713078|-813738.14|quinn garcia
+448342433|4331434.98|calvin young
+809523207|-2954503.11|nick polk
+-616152685|-2534653.62|david thompson
+-1675731725|-2530422.97|alice brown
+202869614|234076.57|gabriella nixon
+-1950862443|-1972072.76|holly allen
+2048285332|-3983451.42|nick king
+-95368574|1032841.29|tom robinson
+665209150|3009532.48|yuri king
+1096973374|-3684269.26|alice brown
+-698411792|-4946050.7|oscar white
+-1651193491|1903730.63|jessica garcia
+-1938368681|1688473.3|zach zipper
+2110108398|4041980.62|zach hernandez
+-517850308|2209822.69|luke garcia
+2112659720|1720032.09|tom falkner
+-920340125|-2918624.41|yuri ovid
+698091944|4884415.59|ulysses carson
+-99742873|-703082.91|jessica laertes
+1258658646|-4642475.92|gabriella carson
+1686530798|-1037135.04|fred ovid
+1547264893|-3853469.99|tom nixon
+228913622|-486843.84|david nixon
+-835615216|2909564.34|mike ovid
+-1930161118|-4759930.21|oscar johnson
+36927920|3799073.08|alice laertes
+-1459435834|528617.16|holly thompson
+671000301|-4904903.29|fred falkner
+1730021052|827026.85|luke king
+-2104258008|-331128.93|tom hernandez
+-1575999014|4428647.41|mike young
+-322261187|-888476.74|tom nixon
+-1432273069|-4844132.34|tom underhill
+486961940|-468067.95|tom king
+1805572098|-2896923.25|oscar ovid
+1927838368|772779.02|quinn brown
+-348625206|912320.57|priscilla white
+1788379108|-2809842.64|rachel ovid
+2075137944|-517229.4|calvin white
+-600333016|115758.34|alice ellison
+-1244393455|-2575680.56|jessica polk
+231811943|-820191.21|yuri ovid
+558405300|-2435551.22|gabriella miller
+-2019539821|4498489.58|ethan brown
+558706209|-3386709.06|yuri nixon
+-676189375|-2336944.13|yuri carson
+888362958|-3065277.69|ethan laertes
+872698079|-1626594.33|priscilla laertes
+1862860346|-1788171.52|fred brown
+-1032054641|1473094.2|ethan johnson
+2108655768|-4524622.46|irene brown
+-1859942373|-2435274.53|bob laertes
+688251479|1451529.95|irene polk
+1516464603|1993874.21|david ellison
+-642031847|-1956483.38|holly miller
+-1427777178|-1979887.06|jessica zipper
+-593172493|-757894.84|ethan robinson
+-455740859|-2573449.63|ulysses miller
+-104834804|2580449.46|jessica robinson
+1002677060|1996930.93|oscar xylophone
+744224509|-3672472.26|xavier davidson
+-1580051989|-4510754.78|fred falkner
+596867449|4689177.5|zach king
+-1026404050|-4320177.9|david underhill
+-1164909280|-2434158.83|fred ovid
+1566573464|-3728740.12|wendy miller
+977690772|-922383.24|wendy young
+1150755920|850044.45|oscar ellison
+627960110|-4396190.89|fred thompson
+-1699744842|-2032155.17|gabriella miller
+-1980999209|1207830.82|fred miller
+-93768797|3822730.25|tom miller
+1791765229|-2707746.08|holly laertes
+-2079142803|3801211.79|victor white
+1081654361|1303685.67|nick davidson
+1176432432|-835459.69|quinn allen
+211380998|-4582268.03|gabriella polk
+2080596761|-2574490.05|nick robinson
+1622817144|-1197548.66|alice ichabod
+-163401054|-4726685.06|holly underhill
+-442368181|1134979.21|nick brown
+1497428726|1449648.6|jessica young
+-2098137688|-2889336.56|rachel young
+-44748020|-2340703.96|rachel steinbeck
+-56058072|4014902.27|calvin robinson
+-1684139769|2917513.79|quinn underhill
+638814095|2830473.17|ethan davidson
+-67382276|2474075.89|bob van buren
+-1267614614|1236118.26|ethan miller
+100566337|-4720560.34|yuri johnson
+1338587501|-2382038.3|david king
+1973170111|3592132.96|alice underhill
+1663067188|-2775761.71|wendy ovid
+893351168|-3215353.57|wendy falkner
+-38917971|4646425.36|gabriella thompson
+-669688073|-2692089.55|bob brown
+-969481151|-3718926.65|ethan thompson
+-525170005|4628386.44|victor carson
+-1830805568|871157.01|david davidson
+356698969|-2240107.33|nick polk
+-1828868008|-3536705.12|yuri miller
+950929056|-4499860.66|holly garcia
+794379844|4492207.31|katie zipper
+-553936437|-3333194.02|yuri ovid
+-381729779|-4596865.36|jessica white
+1203299685|394506.29|ulysses davidson
+1300235094|-1778787.81|alice laertes
+1737815965|3338427.85|calvin white
+141556924|1867536.84|luke quirinius
+-1270118637|2252362.91|calvin falkner
+1272032817|179571.89|rachel quirinius
+684513062|-4473202.12|nick xylophone
+-373479477|-1199391.64|calvin carson
+-784694279|2877981.63|priscilla falkner
+-1226315507|-2041234.46|oscar zipper
+-1129453692|1178901.41|bob carson
+1549713686|-720171.58|victor polk
+-1964336489|2903616.94|ulysses garcia
+1636745778|-2001162.68|yuri quirinius
+1857344426|3048907.64|victor johnson
+-1054630466|350605.49|victor van buren
+-1801329986|-1284397.88|ulysses steinbeck
+527693666|2896297.92|yuri hernandez
+1250038965|-2719696.31|luke ovid
+1321729815|-3083515.83|david hernandez
+-722750974|178224.5|alice ichabod
+1768290153|-2585198.14|mike steinbeck
+267576173|-1826959.23|fred young
+-428160530|-1197562.81|nick johnson
+-1353589404|-4249831.74|alice brown
+1248034300|-813309.05|calvin davidson
+1708594452|1449112.97|wendy hernandez
+1866849786|-4456195.52|yuri steinbeck
+-1780005827|-1166469.81|ethan allen
+-920459813|-324711.48|calvin king
+-1142886536|-693409.55|gabriella steinbeck
+-1407563354|1369892.37|oscar steinbeck
+-1685794708|3671465.1|priscilla ovid
+75140386|2699892.48|gabriella allen
+-1390460876|1635540.24|zach carson
+-1018604036|4954097.28|alice hernandez
+1365989314|-2844705.44|tom zipper
+2020730865|2459700.36|oscar ellison
+227623749|-1968242.11|katie garcia
+-1418221837|-3284499.99|calvin ovid
+2105150125|-3052944.29|ulysses young
+1207770431|-3249891.01|sarah hernandez
+-1703992103|-3898185.26|wendy ovid
+1507322844|-577234.49|priscilla robinson
+-2104337371|-2605633.05|ethan young
+-88389125|-1865580.42|fred miller
+215558730|-734633.6|david johnson
+-1946756055|3431207.75|luke thompson
+811044562|4930284.52|bob garcia
+1086441059|874147.23|ethan xylophone
+-452800895|-4568517.53|katie brown
+81427552|-4341881.95|calvin brown
+192500614|-275590.18|oscar laertes
+-20117079|2058837.32|bob falkner
+132013419|2211.58|david ovid
+-169154721|-4831264.35|oscar king
+-1721359142|2089521.22|jessica steinbeck
+-44431420|4707231.08|david falkner
+1711856345|-1570294.55|calvin garcia
+1883969489|-1476860.06|alice polk
+1524321313|950837.91|oscar ovid
+191964561|-299746.33|alice garcia
+-1887616895|-4342244.19|david underhill
+1060400623|4559192.95|ulysses robinson
+-1651655786|-953183.3|yuri carson
+-1936842954|3785822.07|oscar quirinius
+-676765879|193065.1|ethan falkner
+-2016416388|2513535.84|fred thompson
+1196960193|539026.79|xavier zipper
+776895662|-2932159.05|calvin van buren
+-1877086247|-2901759.36|mike carson
+1395421178|4985372.89|ulysses xylophone
+37231148|2545783.05|jessica ichabod
+-1331007304|-4672324.13|oscar young
+-1065443740|-2956611.09|tom miller
+-1953413326|618938.96|fred king
+2032975686|2555235.19|david miller
+682769886|-1834683.93|nick van buren
+-2016535116|2706056.67|tom polk
+-2083976222|2131469.69|calvin ellison
+-2013273704|-412349.91|yuri hernandez
+878476775|1697569.53|ethan young
+821398902|1623634.94|rachel nixon
+1566086341|-3225744.58|quinn thompson
+-1402849241|2826576.99|tom nixon
+337635170|-4211078.68|sarah quirinius
+-1179271223|-1724524.44|wendy thompson
+1445614028|-3966210.87|zach davidson
+-1429086680|-2218908.05|luke young
+1193110922|-2637162.2|yuri king
+-1555289599|2244884.43|yuri van buren
+1866893202|3625692.64|priscilla nixon
+515559911|-3973231.46|alice quirinius
+-289024414|-1938234.38|calvin nixon
+-163803969|3168615.11|luke laertes
+-117627099|-907647.2|luke zipper
+-1469667075|445557.34|yuri hernandez
+426255112|1676059.76|priscilla miller
+1378066229|3552008.64|tom white
+1437482485|3791098.05|ulysses ichabod
+501343656|-2517731.58|katie king
+-298555804|2597531.69|david ellison
+511068597|3983773.43|luke steinbeck
+-412992997|-3822875.98|calvin davidson
+-1153787506|1189478.19|quinn carson
+993584621|4792940.25|gabriella young
+-1260635267|741591.19|irene johnson
+1306754765|186812.75|sarah laertes
+-475709032|-731183.48|bob davidson
+-587882280|3126007.71|xavier underhill
+-257169474|-3001965.84|david falkner
+1195836704|-426088.47|rachel xylophone
+-580253914|-1446128.28|irene ichabod
+26515815|-2558366.32|jessica underhill
+-2077149711|-3080796.73|victor garcia
+336200960|-676677.04|holly ovid
+-1086467462|-368729.14|oscar allen
+635665551|-210145.05|zach young
+-350785188|-2156689.89|luke carson
+-698424722|4063835.34|zach underhill
+-468198813|-3131508.26|wendy carson
+1914012620|-2894539.25|gabriella carson
+-1261812766|2724144.4|wendy davidson
+573823735|739569.84|mike quirinius
+-202531544|-1089969.73|jessica hernandez
+-2040877758|4136476.2|jessica davidson
+690538505|1259758.6|yuri ovid
+-1255526532|-4083433.01|priscilla quirinius
+1497657014|4711825.5|oscar davidson
+-440687084|3464172.12|ulysses zipper
+951647826|-921722.77|bob allen
+-346241504|-4049143.06|mike davidson
+-1233582066|3995222.48|yuri davidson
+-1364720889|3080802.87|rachel van buren
+250163637|-4657232.94|holly davidson
+330924171|4001384.89|fred miller
+658162664|-2702126.79|calvin miller
+-1098996903|4659615.77|alice polk
+337123194|-2168399.61|bob underhill
+-720321943|1427872.12|xavier king
+1279358124|-3144772.74|rachel ichabod
+1582193117|2804271.75|rachel steinbeck
+1565767231|3459473.29|priscilla nixon
+-1340414946|1581413.29|quinn brown
+-302383212|3731787.73|david garcia
+761215067|3109868.18|tom robinson
+1682071622|1180719.03|zach king
+-1697935927|2204419.35|gabriella thompson
+-2147014375|898995.14|yuri ovid
+-939321049|3682215.48|yuri underhill
+-373797633|2958050.48|david falkner
+983512698|3655241.7|zach allen
+-1203683675|4383092.86|fred carson
+553009919|-4808407.43|ethan van buren
+1683463635|-972611.58|ethan king
+1717519287|-4270966.43|tom johnson
+1771604344|1832012.66|bob nixon
+1065304873|896344.14|ethan underhill
+-1451238854|2927752.4|mike ovid
+275088684|-3053550.3|mike underhill
+-1155287330|-2380661.07|sarah steinbeck
+113203661|3143590.4|ethan zipper
+1023749878|3922821.44|gabriella white
+1089472690|-469269.81|jessica zipper
+-420175476|-2746343.91|david falkner
+1635980432|2390948.12|ethan miller
+317127856|3589980.14|nick allen
+1356497539|-2184609.9|mike laertes
+-666163113|-1175304.66|david allen
+329053041|15215.64|david garcia
+-2000125512|-4090533.45|priscilla xylophone
+-1873623621|1161718.79|zach brown
+514260511|1519065.65|rachel ovid
+212480999|-3729094.03|priscilla hernandez
+-91903212|-2589602.8|rachel van buren
+521799952|-2521088.42|david johnson
+-1068589318|-2002879.83|jessica robinson
+1247026746|-608475.65|rachel ovid
+-383690544|4425478.43|yuri steinbeck
+484242389|4709988.65|irene xylophone
+-194849125|-648436.59|priscilla xylophone
+-1611763423|3616533.72|irene nixon
+347709831|-3280614.59|oscar allen
+-463425747|3972530.26|tom van buren
+-809270267|-4725612.48|gabriella quirinius
+1895679900|972735.22|katie young
+-731258575|1478119.05|ulysses laertes
+-1678542965|4942252.14|jessica miller
+-2111601036|3342028.34|tom laertes
+167187731|4765.41|victor young
+424255270|4512899.77|xavier ichabod
+-1527027087|-3724112.69|alice zipper
+384761737|-3516605.87|bob polk
+-1733586000|868779.93|zach steinbeck
+-414541956|-479157.44|oscar van buren
+-202994585|-2157190.31|oscar king
+1767397047|-3190467.27|oscar johnson
+1116913912|-221892.19|mike ichabod
+920089470|4178046.03|katie falkner
+-312261828|-2053841.5|irene nixon
+1765648878|3238807.09|bob miller
+-994163536|1690425.29|jessica davidson
+2147483017|-1115166.92|mike brown
+1575534716|4956478.64|holly allen
+685873031|4522934.75|quinn white
+-839262738|1292265.26|oscar ellison
+183950351|147470.2|calvin steinbeck
+-1988945870|-4534591.07|bob davidson
+-689468048|4236255.51|calvin polk
+-1582583171|2176652.05|nick allen
+1021845477|3219280.11|calvin steinbeck
+858718175|-1476193.9|calvin laertes
+-1780151912|-503987.24|gabriella davidson
+-407397258|-999716.34|bob allen
+-1291898416|4933915.16|xavier laertes
+468021689|306715.09|calvin johnson
+-1872332901|-3808090.79|holly white
+355619354|-806321.2|yuri quirinius
+588114766|-2814308.25|nick young
+-1836601303|4914364.09|sarah davidson
+515872687|4469117.22|victor robinson
+-380140554|-704483.91|wendy xylophone
+-1058150743|-1368935.19|sarah thompson
+-930585218|3460203.89|calvin quirinius
+-391865647|4374428.4|mike ovid
+-1002943183|-3525777.9|victor polk
+-1264037320|-1369938.33|fred ichabod
+603380724|-4680996.39|fred xylophone
+-1600495572|1778207.1|priscilla nixon
+1865781814|1537665.14|rachel polk
+-286272777|4789062.1|gabriella brown
+2010483180|-3166013.15|rachel young
+-133595190|3856908.91|fred robinson
+1960061323|4436287.33|bob xylophone
+449004105|-2377278.53|david nixon
+270888548|-4945478.02|holly ellison
+1365839484|-4912472.02|jessica robinson
+-1542526231|3625610.73|bob allen
+1399280396|1582852.76|irene hernandez
+-958662214|-2443704.53|david ellison
+1204504334|3485587.99|rachel johnson
+1966932122|3828987.8|gabriella hernandez
+-1199019553|-2890774.53|xavier xylophone
+-911114594|-3835723.54|xavier robinson
+-1856535384|925079.84|wendy carson
+-97446115|2857684.79|wendy miller
+-1769655121|-611227.79|mike polk
+2057987931|-1246047.81|rachel van buren
+-366331265|-4881283.4|calvin xylophone
+-358594829|2798592.87|fred thompson
+165531014|4456630.8|rachel ellison
+-2035438774|2265854.67|bob thompson
+1609356484|-2889586.84|sarah ovid
+1868944583|-2365492.25|jessica nixon
+1103787158|-4730441.28|katie underhill
+1252475020|3868639.24|tom davidson
+2038214348|933144.33|katie van buren
+488317897|2251208.77|ulysses king
+-1840239228|-91948.83|wendy nixon
+1088259602|-198008.97|victor miller
+-1301706113|-997817.56|katie laertes
+1190753860|3811454.57|zach brown
+-1324823566|1440241.65|calvin van buren
+159703500|996255.98|calvin young
+-1523501825|1527804.29|holly ellison
+-2141890434|-1890800.82|oscar ellison
+-595641520|-3125621.58|bob van buren
+-1015214291|-4006503.75|luke quirinius
+36127895|3018276.96|calvin quirinius
+-344337967|1197271.37|nick carson
+1941442017|-1452881.02|mike xylophone
+-1598039585|-904016.71|fred zipper
+959257581|-2827574.68|oscar davidson
+1518057484|1216655.52|quinn nixon
+495034131|95581.43|katie carson
+-1247723318|873044.91|yuri davidson
+452959843|94239.42|david davidson
+1745600781|822316.8|ulysses zipper
+213275900|-4460695.74|ulysses thompson
+-153797303|-785498.12|priscilla steinbeck
+-382910006|3856439.92|david allen
+-2053624164|-3755069.78|bob van buren
+-84488912|995007.74|oscar johnson
+-1333810522|-1203755.22|quinn king
+2063624522|-353926.8|quinn robinson
+-1619709821|2164355.32|sarah zipper
+-171376438|-3952911.49|tom ovid
+-1264645674|1107139.43|yuri laertes
+-1626393195|4787196.8|oscar davidson
+-645572365|-17550.8|ulysses steinbeck
+901557175|-4971371.17|holly young
+1227562362|3781489.03|tom carson
+1978873417|-2808569.68|jessica xylophone
+356106007|1521782.83|priscilla allen
+-1686863032|-3496371.76|fred underhill
+-226190826|-1188450.55|victor falkner
+-1742165850|-3331772.57|katie white
+1459334140|-3136276.9|quinn quirinius
+-1770059789|3497185.53|holly king
+-2053937525|2922671.18|rachel young
+-931254416|-3941437.94|luke laertes
+-1016599139|-1836381.86|victor laertes
+-1392443799|3756236.58|tom xylophone
+577579225|-2159146.81|katie johnson
+1125330981|-1542543.07|oscar nixon
+779496521|-4499320.31|victor miller
+1821826012|2142705.72|victor young
+2094406427|2948920.3|fred quirinius
+1474576738|-4424694.52|calvin brown
+368369217|3762546.85|gabriella falkner
+-1681804911|-389180.06|yuri white
+262300858|1712003.73|gabriella zipper
+-1936566636|-1638110.11|yuri ichabod
+938259854|4325143.21|quinn ichabod
+988767139|891882.26|irene king
+-483494923|-3226907.55|david miller
+-2069195557|4780769.24|bob steinbeck
+-1161518050|2931757.45|david white
+2079667993|-3362231.75|oscar miller
+-1978592366|-3488540.2|katie allen
+528088385|2569442.53|rachel davidson
+1654772258|-2765510.49|xavier davidson
+-773389524|3337314.32|quinn ellison
+-99469085|-424167.77|rachel laertes
+-1213638621|2040273.03|bob thompson
+-2084756821|-1561737.55|david carson
+-1462464047|3639872.55|wendy ellison
+1184220670|-875915.33|calvin carson
+146476502|-3349766.62|quinn thompson
+243409536|1353208.77|quinn ichabod
+274856844|963631.44|mike xylophone
+-850805703|-2385403.41|holly garcia
+-1437738850|-3030514.73|ethan laertes
+511802693|3854850.62|oscar laertes
+-1800992136|442609.4|luke thompson
+-1438410012|-2387486.06|nick allen
+905739212|-2321215.19|katie hernandez
+584275146|-1318688.97|jessica zipper
+-645599087|-4270965.28|irene robinson
+-837753270|-2472888.93|tom johnson
+1544809533|1786594.78|wendy garcia
+-474736975|-960230.67|katie steinbeck
+217111875|114396.84|holly miller
+-1697443096|1033063.92|rachel hernandez
+1650890226|1935229.02|zach king
+271625282|1496212.58|yuri quirinius
+2009939512|-4596194.37|holly ellison
+-1114681105|-583693.18|luke polk
+-1334007028|4611742.25|holly underhill
+1977426065|-102310.6|ethan johnson
+-233654433|2359192.76|holly laertes
+178713590|710580.5|ethan miller
+981903338|-462247.95|wendy ovid
+-1106044718|-2092055.43|holly steinbeck
+-1451418056|-954151.74|bob brown
+1882551044|3458864.94|zach young
+1376286884|-2679171.67|irene underhill
+1448327923|3377457.72|nick king
+-252705691|4646071.75|david brown
+-887800641|-4331543.44|mike thompson
+-848237653|3540150.63|fred laertes
+-1369252220|-1061771.7|alice hernandez
+-309224076|-4840653.47|tom robinson
+1137499117|-2827927.45|rachel underhill
+-1342546504|-4663794.3|wendy ellison
+-54025277|1601596.08|nick carson
+879889384|-2349817.66|ethan ellison
+1004677170|1521374.08|mike polk
+-964244739|-3102483.32|wendy allen
+1674628358|1902164.7|oscar falkner
+1538810249|2436133.69|irene zipper
+709459724|-1196226.48|mike thompson
+-1108692597|-3800347.05|mike ichabod
+1548318289|-4600541.96|tom miller
+1140711149|-1223247.86|nick robinson
+-1290059510|4687155.17|irene ovid
+1753706752|-3740294.42|xavier laertes
+1375012656|-3283011.26|bob ovid
+1559969351|2520386.65|fred nixon
+-341176500|1577774.57|tom underhill
+-2056318090|-2076581.06|zach van buren
+384793302|-2669591.6|luke davidson
+1530074479|1924573.9|nick van buren
+-188709569|2260434.53|zach king
+-674190424|-3203615.76|sarah robinson
+1349606008|2045542.48|quinn young
+-32007373|-4034226.23|quinn ichabod
+-1183487336|3732145.7|ulysses white
+-1688357787|-3046983.17|calvin zipper
+-277543445|-2464656.82|quinn steinbeck
+1988713435|3217671.31|xavier xylophone
+-974546544|2314693.76|yuri white
+-285860551|-2810903.7|irene ovid
+-1265293701|-3097176.17|wendy quirinius
+1337436416|-3529799.91|nick zipper
+1587746144|430432.09|tom xylophone
+594669655|-899162.5|wendy steinbeck
+1316120588|2444321.16|victor ovid
+396546193|1193608.86|bob thompson
+-1823142655|-239473.8|sarah thompson
+-1642807858|-1455078.18|mike johnson
+-683617161|512170.96|jessica johnson
+1543726877|4576809.61|irene zipper
+1056753180|-4294127.28|luke nixon
+1045968867|-4977456.97|yuri robinson
+-284463842|-3504968.97|jessica van buren
+1667838310|-4115824.28|ethan ellison
+1200598103|-813170.85|irene miller
+-1299665707|-4707050.05|victor ichabod
+-418980143|-1206654.42|quinn nixon
+-1539344087|320919.99|mike quirinius
+-112787260|-4383701.09|holly garcia
+-2023274062|4708591.03|xavier davidson
+-47027228|581271.52|rachel xylophone
+-1286874841|4358939.24|ulysses ovid
+-920525408|-1652051.75|katie carson
+-638889327|118923.33|sarah polk
+2026399887|-102018.52|mike davidson
+-596652938|4781080.51|bob thompson
+-675364268|-2257535.6|oscar nixon
+-1467171843|3051131.78|gabriella johnson
+501081480|-2047425.34|katie underhill
+36998412|624429.42|wendy garcia
+762395560|4133627.19|gabriella quirinius
+-1065365095|-1324946.24|mike garcia
+1397449952|3678567.61|holly van buren
+-1238102644|3158459.95|ulysses laertes
+1253719115|-2944863.54|rachel underhill
+70440469|821910.33|tom zipper
+-3230436|-1274444.05|mike nixon
+-989259189|-4718519.01|wendy quirinius
+2034485636|-3030056.31|tom thompson
+1349765205|3171149.28|katie davidson
+1955090066|885338.37|oscar ovid
+-11276890|-3137713.54|quinn ichabod
+1903861809|4006355.79|wendy hernandez
+103135712|2359696.57|irene nixon
+834981643|-1955472.23|ethan xylophone
+-1110457465|-4959474.23|jessica hernandez
+-871115254|-351758.65|bob carson
+1419403911|-2990166.49|luke white
+1236103065|-2768853.21|katie zipper
+598816819|-879922.82|luke allen
+897383183|-2532148.96|alice young
+-708977321|2916156.86|jessica zipper
+289444017|243366.03|yuri white
+801930841|4079393.35|sarah brown
+262240083|2565394.42|xavier johnson
+377993918|-4627221.59|ethan allen
+-2062174066|-4752400.63|zach king
+1167439960|-2140088.8|irene king
+-1260904208|2844671.88|quinn xylophone
+-455748604|4946971.44|fred carson
+-1288790074|2112572.74|mike allen
+-1976762159|-1699556.2|yuri young
+-1183824318|-717428.74|luke johnson
+-47019036|-903487.26|jessica steinbeck
+-407820244|3076434.47|xavier laertes
+-1701646739|1028014.68|rachel ellison
+-1468684186|4334682.81|oscar nixon
+810641543|3660392.3|tom young
+-1373873689|2564930.65|jessica robinson
+639213293|-196378.26|alice miller
+-669048330|4170990.05|tom brown
+-1232981329|1295508.7|bob nixon
+-1248017710|3998329.47|katie thompson
+-1905683078|3108442.98|rachel thompson
+456390148|-115362.22|david ovid
+-130284008|-2151672.81|irene davidson
+-1412246507|3620098.94|priscilla laertes
+339802149|2339286.88|bob ichabod
+-1966603432|-2288011.47|holly steinbeck
+326041391|-2003109.98|xavier polk
+1743264260|3597024.68|gabriella falkner
+1487861952|-2597339.98|zach xylophone
+658462693|-2088865.9|ethan xylophone
+1053061628|2425425.65|david laertes
+-1636406728|-1307047.7|alice young
+758412513|-860558.52|luke ichabod
+483778705|4755057.71|quinn ellison
+1939527240|-4454754.57|alice nixon
+2143708139|2989855.48|mike young
+891689267|-3394669.28|holly polk
+-2026884826|-3690178.03|rachel ellison
+2124141841|4189533.82|quinn quirinius
+1669914264|-1327581.26|wendy zipper
+-1490316288|2348241.05|david king
+1425973394|-3288682.47|ulysses zipper
+1787306800|-1282327.99|katie carson
+698950355|-3473513.0|mike brown
+1241686778|4378658.25|katie young
+-1732496025|-3331175.6|oscar allen
+-1031924408|2092275.41|rachel miller
+-2056517274|4217024.65|sarah hernandez
+1526990928|-1717559.85|alice nixon
+307175053|271240.64|jessica falkner
+768940272|4358313.51|alice polk
+-1593680724|765848.98|quinn davidson
+2055975665|2981113.03|victor johnson
+-150424500|-67600.01|zach davidson
+-1729528029|-2228593.31|jessica ichabod
+-1438443471|4070308.66|yuri van buren
+-1584093669|4949690.65|priscilla thompson
+376913273|2784258.56|ulysses steinbeck
+1495352294|-4101700.47|priscilla falkner
+-5479799|-1613028.55|mike ovid
+-271291353|-3452583.56|rachel ichabod
+317453566|4028459.66|tom quirinius
+1212923708|1246984.19|tom ellison
+357588334|4013719.16|fred ovid
+1235820694|-2362624.2|ulysses xylophone
+-658240622|-2075787.25|calvin laertes
+-1451005112|4371131.77|wendy laertes
+1446324256|-4027374.93|victor thompson
+75521374|-3654842.42|david falkner
+665559468|-821373.86|luke ovid
+690629567|521274.11|sarah ovid
+1848487930|-673436.75|oscar hernandez
+-1774413966|4956928.56|katie thompson
+1903556051|4688440.83|priscilla miller
+2142618263|1465204.48|ulysses hernandez
+-1802818734|-1674627.01|mike polk
+2076817039|3249437.2|ethan king
+-1694088441|2752103.34|ulysses brown
+621501932|2347723.51|david miller
+-948086126|-4687971.85|fred steinbeck
+2076767309|1488348.73|ethan davidson
+425790339|-1860383.03|calvin brown
+1029598001|-225713.27|luke davidson
+-322311270|4133844.44|alice polk
+2134855268|3780448.32|fred johnson
+1066781632|2604700.8|katie zipper
+170400665|4647203.57|oscar ellison
+-1661887657|-4424750.79|oscar van buren
+1925257278|4116898.81|nick van buren
+1577406887|1598292.89|gabriella ovid
+-283490875|4713434.74|priscilla thompson
+-198970977|-1597256.96|katie johnson
+-906987585|-660081.01|oscar steinbeck
+282670099|-4138063.32|tom white
+-1389784915|-1049873.99|quinn robinson
+919122766|-3696148.25|alice nixon
+747362640|-2711275.83|priscilla robinson
+109913738|-2769585.37|nick polk
+707793874|1561943.17|luke ichabod
+-668440922|-3906693.72|wendy carson
+-1426282703|3760858.02|victor ichabod
+-663542467|-939779.45|rachel hernandez
+-1283794779|3830924.41|tom van buren
+-1894021661|-3196652.89|jessica nixon
+1201000445|-2265591.2|xavier polk
+-1168132803|-4852082.08|jessica brown
+-1844319951|-223415.14|jessica thompson
+-1998439394|-1108364.42|sarah davidson
+616360065|1951060.31|alice garcia
+-1133792002|-4547304.22|quinn king
+1509152783|-2423217.24|oscar miller
+2073872482|1979216.34|calvin thompson
+-967838533|1719243.47|alice davidson
+-969574128|251833.49|priscilla allen
+521334155|-4912470.78|victor van buren
+-1165040876|2930519.44|david davidson
+1627202604|2264343.73|alice young
+-24956646|-3883600.96|yuri nixon
+-1662818797|295384.14|calvin allen
+2091303295|1334468.35|tom ichabod
+-1942942358|-2423751.19|xavier xylophone
+-522399462|1096161.4|priscilla xylophone
+-50893343|-2054030.11|fred miller
+468358077|-363924.61|zach quirinius
+-33868575|-2725494.29|nick xylophone
+887356123|3264972.94|nick polk
+-809581080|2316843.11|irene davidson
+682989774|1365911.93|katie ichabod
+2064640742|3162761.31|holly xylophone
+-1927502921|-3962898.05|irene nixon
+-1387971070|4636633.29|bob brown
+273687794|4653739.55|luke young
+1528941826|1855502.52|mike davidson
+-54431090|-3123754.35|sarah young
+-201559975|-2917123.88|ethan quirinius
+-1179896246|3994334.87|ulysses hernandez
+-93555074|1953656.21|tom underhill
+-1436347898|-1143941.34|calvin hernandez
+108685844|4799979.8|nick van buren
+-1490390097|1028835.76|xavier steinbeck
+129313833|3887370.74|calvin xylophone
+-1749945596|-613430.34|alice young
+-136303288|2741654.56|katie nixon
+-947801648|2853235.96|priscilla johnson
+-1182953615|2744046.29|ethan brown
+-231851158|-1093843.16|calvin xylophone
+-1366280170|-3771949.55|oscar underhill
+1819630225|2475159.29|quinn falkner
+-924343136|1510925.12|luke davidson
+215080208|-2216110.18|tom garcia
+2000583975|-2620323.93|nick steinbeck
+-1170910557|1887708.97|xavier zipper
+1545189491|-3618438.09|yuri falkner
+-2074275576|-369150.18|fred young
+-865044141|4604552.17|david ichabod
+139213150|660713.33|david zipper
+1477177572|1645283.57|zach van buren
+1191031565|-2050910.94|oscar miller
+-1585622397|-1777619.85|sarah carson
+-215539993|-3220468.91|calvin ovid
+2043712817|-1416917.41|katie brown
+1087781196|4039512.36|xavier xylophone
+-515691539|2353915.71|luke garcia
+1975782526|-3027060.4|calvin king
+-52733651|-2539215.13|oscar van buren
+1102136408|2550238.27|priscilla ovid
+-1990714507|-4776875.27|zach van buren
+473103143|-783539.13|calvin polk
+-439666744|713056.55|sarah hernandez
+-2127363225|-3601250.37|alice ellison
+129657947|-3054299.51|jessica robinson
+-477162370|4196487.47|nick steinbeck
+-2096781089|-2334119.26|ethan davidson
+-1504057135|1186954.88|xavier steinbeck
+1546459948|4235530.99|nick ichabod
+1898731985|-3036795.58|rachel hernandez
+-1656144313|410310.03|quinn steinbeck
+-782472502|-3990522.34|nick ovid
+1019393768|4869861.75|nick johnson
+321392009|-3055230.83|bob garcia
+-1115622377|-2179900.43|jessica brown
+1143589980|-187847.16|mike white
+-807791759|4427094.49|katie brown
+915962551|-931538.57|zach quirinius
+-103759819|1147513.84|calvin miller
+-1974981973|1488659.65|victor polk
+-767723213|3055035.6|priscilla laertes
+1166210808|1532251.78|bob johnson
+-716231389|3426194.4|irene quirinius
+-1933948316|-4273955.08|ulysses johnson
+2137931287|-1257648.85|fred polk
+-423282032|-4643084.4|ethan white
+-614003702|4192240.2|yuri steinbeck
+-1371283067|2460910.54|xavier falkner
+-1045695571|4746138.46|mike nixon
+1988192962|-3742360.22|gabriella davidson
+78612998|-3716896.33|holly brown
+-1207215788|4642336.22|fred hernandez
+-264792433|-4938222.84|mike laertes
+-325422476|2606987.64|irene young
+1852322996|-1989191.24|bob ellison
+444351640|-10989.67|zach van buren
+1106275099|1789217.84|yuri polk
+-760707838|1788805.11|katie robinson
+452049402|-502631.2|quinn king
+1655571996|-2530873.95|tom laertes
+-1846447273|-772007.2|ulysses garcia
+163833696|-3495378.52|nick laertes
+-137138409|-1084114.04|quinn xylophone
+229532519|-2566305.18|oscar brown
+-930371044|-109367.93|irene garcia
+-187278473|-1501475.13|ulysses nixon
+945875689|2654168.24|victor carson
+1630011316|-3472886.72|priscilla zipper
+1041919875|-676269.43|victor quirinius
+-1005467207|-1409671.78|jessica steinbeck
+-1306378364|2706067.85|fred underhill
+674298680|-268079.47|oscar brown
+2080166553|-2275722.63|calvin robinson
+-1812658938|3195789.66|yuri king
+-1661165838|-3059856.74|yuri miller
+601764500|3945278.65|fred king
+-2085116866|3486418.68|ethan underhill
+1814938817|981164.63|sarah quirinius
+1006663494|180441.95|jessica hernandez
+1707334294|1319128.77|oscar garcia
+304709464|4754721.0|quinn garcia
+-622251722|1730649.07|luke ovid
+670388712|488706.42|oscar quirinius
+1595175991|-1070704.64|bob allen
+1846227521|1132121.56|mike garcia
+2114100773|-1348723.04|mike hernandez
+1769669148|-266610.69|oscar johnson
+634606801|1094861.79|rachel ichabod
+586429806|-3674259.9|luke polk
+-1355133004|-3578736.34|priscilla allen
+1092808403|-3115460.69|oscar brown
+-1160757599|-4509502.32|jessica polk
+1845762910|-263030.19|ethan allen
+1098147187|-505906.68|irene quirinius
+1870365622|-2255346.31|ulysses polk
+589153659|-3057587.76|quinn davidson
+476521926|-2067093.73|david van buren
+-1389213745|-1839964.07|quinn davidson
+1491801455|4617501.61|victor polk
+380402642|-1946684.91|nick young
+-1620218885|-3712897.44|luke xylophone
+-1337511012|-3733798.73|yuri polk
+1621542201|2176970.33|luke miller
+1072243690|2362015.16|xavier garcia
+1125727902|3655496.81|irene quirinius
+-1703755781|2401396.13|victor davidson
+-162713684|2317189.41|tom quirinius
+1844207960|-873341.92|quinn ellison
+-870140786|4158535.57|alice robinson
+-1135445388|1827400.51|zach steinbeck
+-906759487|-3321687.56|ethan xylophone
+1211541179|3763419.93|nick steinbeck
+2125696264|-3532846.25|bob underhill
+-2109868719|-3982978.48|sarah ichabod
+1846171433|3827896.06|alice young
+-1604625474|929819.21|victor miller
+1355797282|2526259.67|xavier miller
+526312503|-156626.64|zach falkner
+-325932840|1558079.06|sarah miller
+1067611979|-1112492.26|mike carson
+532475499|576352.98|mike thompson
+-1729027317|-1775028.81|alice robinson
+-652277052|-1400255.25|calvin garcia
+1422395922|1506031.91|rachel van buren
+1610747099|-3572736.08|yuri laertes
+-464731389|-2693130.83|oscar garcia
+-1711943168|4266313.74|david underhill
+2094888253|4687948.17|zach quirinius
+1360138753|-1933114.16|sarah davidson
+1718516233|3457421.35|tom davidson
+-1274760189|-3524722.69|irene carson
+-769111698|-4563093.12|katie ellison
+294680428|1315069.37|fred ellison
+-681504679|2860251.68|tom thompson
+370758703|-4547691.07|fred ellison
+-1192293894|-3197283.29|priscilla thompson
+-2068854182|3915196.7|rachel steinbeck
+-1854625710|-2872610.56|calvin miller
+-1150223193|1741799.7|yuri garcia
+213143654|-211455.28|gabriella king
+899789469|-3656250.89|quinn young
+30866590|-4033288.35|luke robinson
+-1688695217|-2686283.19|priscilla carson
+1420626985|2481623.05|bob white
+-524964138|4955560.1|luke hernandez
+-131993461|-3211961.88|ethan laertes
+512291875|-701700.03|fred hernandez
+1015470747|-2624798.97|oscar white
+64287179|-4292698.38|victor garcia
+1125387934|-3520599.94|ulysses ellison
+711697852|-1230447.4|quinn carson
+1796715324|4509828.24|yuri robinson
+2126974250|-454277.69|priscilla polk
+-1944006272|1148853.93|mike miller
+-104987359|-4048229.69|jessica ovid
+596665387|1818958.73|tom hernandez
+2018792806|-2864365.52|david allen
+1889030|2097190.44|tom underhill
+945923017|-2999757.54|gabriella miller
+-746087706|-4721343.07|alice king
+892349157|-4445227.8|bob thompson
+-1422115423|-2178802.24|wendy carson
+1087027960|2211280.37|katie ovid
+-1326746306|2213940.2|rachel ellison
+-2116643725|1111306.67|victor king
+-1071113239|2572731.95|ulysses thompson
+-1951571141|-2904866.83|sarah van buren
+-86255972|-2296161.28|luke ichabod
+-1636444444|4851647.2|luke hernandez
+1733911170|-2488571.8|tom ichabod
+-914008260|4489164.48|bob van buren
+-270533255|-528829.79|ethan ovid
+215348743|3521567.74|rachel robinson
+1668206650|4352907.87|luke van buren
+815484362|932646.98|quinn quirinius
+1997935897|3177641.32|jessica ichabod
+691358422|2716820.91|katie ellison
+1662320732|-2336998.7|sarah miller
+209598252|4347201.54|irene johnson
+690208650|1625278.4|wendy polk
+2038602451|-1701876.74|tom garcia
+1872120661|-2416232.5|calvin falkner
+-591009371|2990530.12|sarah garcia
+1829950430|-3834814.1|fred miller
+662330564|531359.19|rachel hernandez
+-1836953205|2269653.84|mike hernandez
+1805650242|341681.14|mike underhill
+-853160795|1526808.67|ulysses falkner
+590747285|-4877553.69|irene ellison
+-1286386568|-898547.67|victor quirinius
+1616852716|4543409.51|alice thompson
+1280025259|416563.21|luke falkner
+435290249|-283327.58|mike falkner
+1372160523|-3260019.91|fred nixon
+1648874154|-875233.95|katie xylophone
+645074938|-2576019.55|victor robinson
+1366186403|487345.46|oscar underhill
+1092843075|157590.72|gabriella garcia
+-479929034|-1347862.5|rachel allen
+147945226|3065197.47|bob allen
+-794879661|4204932.23|luke falkner
+-576473339|-2710584.78|jessica zipper
+-309672766|-4249423.93|rachel white
+-209589690|4955325.29|sarah garcia
+865462883|-1694109.75|wendy thompson
+2000379453|-3807911.7|sarah falkner
+-1542117109|-3515829.62|calvin ellison
+-672329788|119232.64|irene johnson
+1878498083|-2285738.44|yuri ellison
+1274193469|3368841.81|wendy steinbeck
+-1096818018|-1387065.78|david garcia
+-546120380|-1970299.66|priscilla ellison
+1780444883|-1153314.01|jessica davidson
+2030513233|-2031805.63|xavier brown
+-1153771159|826491.77|victor laertes
+-2011994008|26544.43|quinn nixon
+-158335919|784447.58|priscilla ellison
+-1393443421|-890261.33|alice garcia
+1688434307|2107731.91|wendy van buren
+1637107508|-4037241.94|nick falkner
+414877187|2049880.12|tom zipper
+618383948|-1175822.03|quinn polk
+304463167|-4837642.11|yuri ellison
+-652460855|1105308.86|sarah hernandez
+-2068121849|898067.52|bob van buren
+-4269374|-4326005.21|mike xylophone
+-1909827727|-33614.86|quinn king
+-742969763|-4930969.86|rachel king
+-1901596264|3241173.12|bob young
+783721312|1672900.06|calvin brown
+895885559|-3619593.03|sarah davidson
+-1108829059|3680886.93|rachel allen
+1690540737|-2931441.66|quinn nixon
+523801168|742213.83|ethan miller
+-832897389|4247869.61|luke king
+-1885135738|4676361.27|tom ovid
+997396296|-4088502.51|katie thompson
+-1624640605|1695021.71|gabriella ellison
+111217639|-3930558.84|gabriella zipper
+753372768|-1243887.38|priscilla van buren
+-1482183211|-1979098.59|ethan steinbeck
+-1099954265|4157538.19|wendy king
+-192779057|-794661.92|priscilla quirinius
+-1674114434|234506.85|priscilla falkner
+-409558430|-2432080.94|wendy johnson
+-2083281010|-3576458.96|oscar polk
+2131971549|-575545.49|zach brown
+421970961|1965899.89|rachel brown
+393764583|522234.47|xavier miller
+-164867008|-37275.5|rachel polk
+5955641|-4340660.91|xavier ellison
+-486188141|-167713.32|irene brown
+30600517|686705.44|katie white
+620546942|4106805.02|holly underhill
+-1147577187|2049246.36|xavier laertes
+1507679602|4428412.13|irene quirinius
+1170687602|1538843.23|ulysses ovid
+-952271044|4726405.48|nick ovid
+276453793|113171.72|victor steinbeck
+259582235|-4982235.19|katie zipper
+1127946982|1512397.0|priscilla zipper
+-1841329352|3864095.17|zach nixon
+1783759191|3388225.28|oscar young
+-1432437716|903866.12|oscar brown
+-1715095482|4702626.11|katie ellison
+1335690462|150176.63|nick garcia
+1402925572|3636926.51|david steinbeck
+416357873|2333503.99|oscar white
+-180357041|4847106.57|katie nixon
+-833502880|-2787660.57|oscar steinbeck
+1764873785|1154773.96|gabriella polk
+-1969002713|408370.11|xavier nixon
+1996252418|-1935734.17|bob falkner
+1117384207|4481348.72|wendy quirinius
+147783702|-1474721.22|ulysses van buren
+277029359|-1565978.93|quinn young
+-969015783|3062164.45|holly quirinius
+1347557119|44455.6|quinn zipper
+-199388193|3945811.44|wendy steinbeck
+-1781137900|-4895975.74|bob ovid
+-490648672|-3708017.5|nick young
+2123266010|2226910.81|luke johnson
+-175960641|-4099884.46|zach laertes
+516822392|-954851.96|yuri zipper
+1381280245|2339616.11|zach brown
+-147475707|-2920715.19|wendy garcia
+677457625|-758045.54|gabriella ellison
+-23970588|-102160.5|holly van buren
+1943141095|-3656116.79|victor ovid
+1558006512|-4927548.83|ethan zipper
+1817157015|4782593.45|ethan nixon
+-1617928976|355081.32|victor robinson
+1468868649|-1386029.52|bob falkner
+-365659709|-2819973.35|nick nixon
+-493238993|1189945.43|fred young
+323227470|-2269140.93|zach ichabod
+-726129927|1198564.05|zach davidson
+-1825291482|-1244513.14|mike nixon
+163889265|2700240.37|oscar zipper
+296483903|694480.29|ethan thompson
+-174699250|-165012.47|ethan johnson
+928111725|-2247459.31|xavier underhill
+325606359|-3051219.91|irene johnson
+928298063|4025195.26|victor van buren
+-1226934587|1472760.75|victor robinson
+793954031|925855.77|irene miller
+-1334215985|-376770.81|holly thompson
+59245139|568053.8|zach ellison
+1520164493|10184.12|rachel ovid
+1331706015|-313167.65|wendy ichabod
+-1769771965|1953233.71|tom miller
+1796469156|-1819478.33|victor miller
+-1048846957|-313514.87|nick johnson
+1624354539|3746528.03|xavier allen
+142146152|939967.7|ulysses garcia
+1467810973|242652.04|xavier steinbeck
+-671142485|4011128.2|irene hernandez
+-987969576|994493.95|jessica thompson
+1593655759|3208249.98|tom carson
+1787516753|372889.89|tom ellison
+-860162946|-3444775.13|oscar miller
+1722465625|2146396.88|tom falkner
+-645424215|1285664.41|ethan young
+591047313|-3823041.3|jessica garcia
+-1963524104|340481.59|katie van buren
+-213770505|-2578174.77|wendy van buren
+1792354132|-4325077.22|david zipper
+-1888906466|-1792432.86|gabriella carson
+-1765548046|2995424.96|ethan nixon
+276669853|4232312.19|victor carson
+49659695|-671454.59|xavier davidson
+225724712|595456.95|victor robinson
+-1906521825|1957142.58|rachel young
+1672428995|1882810.5|wendy ellison
+1730765979|-56433.69|rachel falkner
+-25963502|13919.99|tom ichabod
+361655603|-4476390.39|xavier xylophone
+-707746880|4400822.85|mike brown
+-1762234544|4096913.42|ulysses quirinius
+1700957792|2464420.63|tom laertes
+204289114|2844729.88|irene carson
+77414|1558082.77|yuri king
+1033898976|-288365.17|holly ellison
+1026244094|-255803.01|zach zipper
+-1784511527|-3588855.77|calvin carson
+-1882545173|2160797.63|ethan van buren
+-1162881647|-3903744.72|rachel nixon
+1072184504|3020260.24|nick steinbeck
+2122448380|284040.47|wendy zipper
+977089775|1945458.43|luke laertes
+-354642089|-4905981.07|oscar brown
+412380406|2549391.51|mike van buren
+-80943285|3325777.59|sarah white
+1680144878|586296.91|wendy polk
+1601929261|1319120.34|yuri hernandez
+-1368918344|4830631.49|holly laertes
+17505512|-246995.97|calvin brown
+2053738190|1024579.05|sarah quirinius
+-48736606|2490510.7|victor brown
+-900790063|154194.0|sarah brown
+-1207023542|-3752255.17|rachel carson
+1783802047|-2335229.01|gabriella ellison
+-1257474153|3291507.35|oscar king
+-1550247323|136292.21|david falkner
+-1119011816|1028047.77|bob polk
+875741786|4012650.03|calvin davidson
+-248271780|3619996.0|bob brown
+-2056558425|1139371.09|ethan garcia
+1590339174|2192352.91|tom king
+102447606|1588863.93|yuri garcia
+1467032559|-2471986.7|sarah zipper
+1391045838|1926771.51|ulysses johnson
+1026262038|398675.73|nick king
+368803404|-59780.93|ulysses king
+1406540689|-4596802.47|quinn xylophone
+145507928|3163737.01|zach davidson
+662634887|702610.26|rachel quirinius
+-584637105|-2739195.36|xavier carson
+-1222180210|4184013.38|sarah van buren
+-952412796|-154506.49|katie ichabod
+-930958204|4932457.32|tom king
+-923465404|-4534609.7|nick van buren
+326313285|-1364441.96|david van buren
+-2145222134|2019211.08|rachel falkner
+1094933352|-4805396.52|nick brown
+-1355036633|-4063429.56|luke polk
+213265278|-1703343.52|katie carson
+369715738|2600075.59|ulysses nixon
+-472150634|440510.75|victor hernandez
+-1205976767|-1028176.65|quinn van buren
+-817236499|-3160881.35|xavier young
+-464456288|-3788717.01|tom zipper
+1738475799|-2955035.78|jessica van buren
+858057987|3032711.22|alice zipper
+1418372864|2797476.86|priscilla johnson
+-196736595|-928900.34|irene king
+-1887794796|-4806042.79|yuri polk
+1246178089|-939680.55|katie zipper
+649454320|-2682987.03|holly nixon
+558258245|-4506464.32|jessica davidson
+-386575242|803910.21|irene underhill
+1820975885|651925.47|sarah van buren
+1392612102|-2542062.47|holly zipper
+-1971566507|1767732.86|victor white
+122873433|-3470287.38|priscilla falkner
+-378072503|-3954249.88|priscilla young
+459651117|4256463.15|gabriella hernandez
+-1537527317|-2541295.91|quinn nixon
+403729849|98984.86|rachel robinson
+-1188289136|1354182.72|rachel falkner
+2078283961|-2904762.8|wendy ellison
+906324661|-1328847.69|victor hernandez
+2107834072|-2740625.77|tom young
+408106955|3010536.73|gabriella johnson
+-2142251657|-1264070.29|ethan ovid
+2072108737|-2099924.35|oscar xylophone
+137407726|4663760.68|oscar ellison
+374182778|-1891610.23|oscar van buren
+-878038540|4719227.82|calvin brown
+1834955123|2836806.58|ethan white
+-1690307434|3257850.07|bob steinbeck
+-1268510139|4342842.77|priscilla hernandez
+779688975|2689332.78|katie ellison
+-178871613|-1794132.99|katie king
+1883644787|-517587.83|gabriella underhill
+-650365215|2205301.64|quinn young
+747585219|-157694.53|wendy nixon
+1039217846|3394124.78|bob thompson
+2044111143|2758769.75|ethan underhill
+1515658537|4502093.34|bob hernandez
+1340386354|3265027.63|fred falkner
+-1386361561|-3464460.73|gabriella ichabod
+-1489339015|-2714795.35|jessica nixon
+1212259451|-3804419.21|calvin garcia
+-471700753|-1097346.89|ethan king
+-774942803|-3212637.84|tom davidson
+1302857660|1721605.85|ulysses robinson
+-1348781996|-1993610.01|calvin carson
+-2143128947|589483.45|oscar robinson
+605940716|-1049157.69|alice ellison
+-663669881|-4438413.96|zach underhill
+729004838|2637611.72|sarah ovid
+-1068178581|3667229.22|holly allen
+474422278|3243849.37|mike white
+226926227|-1839174.27|yuri white
+452302720|3325841.67|nick laertes
+-902658624|-4294620.45|quinn zipper
+1862537048|2035706.83|david nixon
+-723252234|-1123290.92|xavier quirinius
+-1893373086|1597880.99|gabriella ichabod
+1416092643|4584411.93|priscilla ellison
+877034872|535990.59|xavier johnson
+421234816|1540284.22|gabriella davidson
+871186899|-1507213.93|david white
+-1489808415|-3598900.94|luke brown
+809508209|-162069.01|nick miller
+1115810933|-3416147.46|mike white
+-1946915120|442435.71|bob king
+1942774841|-652776.02|mike laertes
+65617212|-4879066.14|mike xylophone
+649635437|1236458.19|xavier white
+-416603822|4804328.55|wendy young
+529571832|-3216634.28|priscilla ovid
+-486066815|-4798063.7|victor ichabod
+1622539144|-4980650.04|zach robinson
+-88930374|-1508862.98|xavier king
+1839664637|2406579.47|mike thompson
+-860113522|-303085.0|david carson
+-1556207401|662014.84|rachel robinson
+-64183707|-3005421.05|alice xylophone
+-152481913|1767034.99|xavier zipper
+-1662377065|-1264303.88|xavier zipper
+641536516|1948275.77|zach miller
+-2069159796|-3422872.24|oscar garcia
+418210297|-1159991.2|holly thompson
+-1870623441|-3892145.11|david king
+668559690|916595.82|fred ichabod
+1323754330|3693419.94|david davidson
+2032273401|165816.27|oscar johnson
+-1322392570|-2233732.38|irene young
+565148820|-601721.38|yuri thompson
+-2097609900|-2537862.86|david carson
+-1152178775|-4165165.79|quinn laertes
+-1232633116|3357867.6|wendy ovid
+-1163717648|-205409.61|quinn brown
+-1372032026|865593.38|quinn steinbeck
+-86336455|-235031.14|luke steinbeck
+659830500|2687186.7|alice thompson
+1130695990|-924407.05|wendy garcia
+-1711562423|-2596052.16|alice zipper
+511537336|-2914216.6|xavier johnson
+1172523252|2264915.46|xavier nixon
+945987836|-1369939.02|quinn miller
+-431512421|3238344.54|jessica young
+2008828444|4641779.59|wendy zipper
+-1304280178|-912686.87|priscilla quirinius
+-1936098115|2301399.95|rachel young
+-266835392|1955554.54|priscilla king
+-898526578|-364455.04|calvin young
+-989040463|987924.73|fred quirinius
+-2030232000|796875.47|zach thompson
+204170868|4477816.92|mike king
+-399063053|-928104.02|gabriella robinson
+-910996447|974541.51|victor falkner
+-1929962418|3838350.14|bob miller
+1230025781|296245.75|wendy robinson
+1467336638|-897215.58|gabriella brown
+1069508069|-695745.71|tom ovid
+156584087|1707490.93|holly carson
+-1501775278|-3936156.12|oscar ellison
+-276684291|3619354.25|victor allen
+-1276808959|3269121.64|nick quirinius
+-1662134556|-3903799.99|holly young
+862368458|2053043.93|calvin thompson
+484032408|-2154344.28|bob allen
+472087146|4094487.32|nick hernandez
+1050867065|866724.35|quinn van buren
+-860966798|-2560264.05|yuri underhill
+1643866095|-2544592.93|katie falkner
+-1313962924|540160.24|ethan miller
+-528710962|-2987843.87|sarah ovid
+-930087230|-1333357.79|wendy ovid
+761065880|1399858.32|quinn miller
+1216938392|4009094.4|mike johnson
+-2138583954|1937236.41|gabriella underhill
+-850724867|-3921883.7|holly white
+-617469323|-3413978.13|gabriella laertes
+459730308|3695203.61|priscilla van buren
+1397916460|1395660.56|holly ellison
+1626582404|-2110821.07|david laertes
+-1792490763|1508805.93|quinn king
+-481526836|-2253405.51|ulysses robinson
+1952927888|-3138655.07|xavier miller
+1500577403|-4394190.47|nick miller
+708626601|490185.66|priscilla falkner
+291019971|-2993370.38|zach quirinius
+1819261304|1708992.0|tom van buren
+-713951075|4971894.48|alice xylophone
+680819729|-770511.69|calvin allen
+-827534876|-2234581.58|zach ellison
+-37209582|4859175.4|zach young
+-1707761828|3070242.45|nick polk
+-1037464159|-319081.54|luke white
+1855983233|4469904.95|xavier nixon
+1286129403|124198.02|quinn young
+-1200392899|-2491082.11|priscilla miller
+-433517043|-1048876.52|bob ellison
+-1432182584|-418242.5|zach king
+1355095990|-3413953.48|calvin king
+-222207119|2806768.38|quinn zipper
+-1683841571|3898162.72|xavier zipper
+-240296201|-2962408.14|quinn miller
+-390750637|-746486.17|wendy quirinius
+599645109|2749134.09|gabriella nixon
+-107304908|3704712.9|mike ovid
+1862702303|-3926741.78|priscilla underhill
+751988823|-2117620.68|yuri steinbeck
+-2008330129|4180779.69|yuri ichabod
+203076825|-4972518.24|yuri white
+164851101|759317.05|katie nixon
+883975409|4194564.33|david thompson
+1821138688|-3528714.63|fred hernandez
+-793895540|3870128.23|oscar miller
+1486149015|-3586498.15|tom miller
+1231010911|-3612691.02|mike ichabod
+-145864591|3730426.56|wendy laertes
+102281859|410766.1|rachel xylophone
+1838354613|281742.96|victor zipper
+69903518|3463286.09|mike brown
+963220538|1238164.15|david white
+1849694794|-4696626.38|yuri ellison
+194740449|404786.24|alice laertes
+-1759883438|-2187430.63|rachel laertes
+-476645004|-1074787.11|victor allen
+-42469187|974911.99|zach hernandez
+-1987028799|2523910.29|jessica steinbeck
+389991667|1006647.23|nick thompson
+394848801|-996042.92|katie allen
+988031324|-989816.78|rachel brown
+-1218300238|1056804.79|david hernandez
+701962559|-3113889.48|victor allen
+1325948121|882693.39|ethan steinbeck
+1236506841|4551702.04|bob brown
+-60824569|1642148.44|katie robinson
+-1438382881|-4206116.81|gabriella brown
+-945966949|-1355496.34|priscilla garcia
+-36599327|2783129.23|calvin garcia
+-1516718975|1276396.13|oscar robinson
+1751771218|875300.3|quinn robinson
+-650852630|-3050050.69|jessica young
+1469206819|-2103116.31|wendy ovid
+-1348845039|-1364921.18|yuri ichabod
+-754299869|-520892.55|ulysses garcia
+208515117|3013435.58|holly carson
+1813238953|-4302068.68|calvin johnson
+-111509299|4488615.87|priscilla hernandez
+-2131758520|1187451.86|xavier allen
+942245451|1058472.98|oscar xylophone
+1254709848|1238706.6|xavier thompson
+-1293719760|-2886936.02|ethan ovid
+63237334|-3690203.33|xavier davidson
+-811773562|2597866.47|oscar miller
+-684363327|-969325.89|zach brown
+1905632917|3901960.08|gabriella underhill
+424914413|820809.29|david carson
+1628115855|200544.77|wendy ellison
+-680786646|-3304054.04|tom ellison
+-908840970|867783.88|oscar thompson
+822688995|2824790.32|fred garcia
+-210545421|3755504.27|mike carson
+-136716832|2011154.71|irene steinbeck
+-891125535|960648.05|bob quirinius
+-988808020|2402014.23|fred hernandez
+-178270042|-747860.95|tom miller
+440281020|-3397295.88|holly polk
+-1577847622|-1073551.66|calvin hernandez
+1434955656|-2057914.38|rachel ellison
+580000233|-1139237.73|mike ichabod
+-1677869065|-2131401.75|priscilla xylophone
+-585460170|294450.48|luke carson
+-226974729|3196467.36|nick garcia
+-1608034203|3681848.88|david johnson
+239033130|-337951.36|holly miller
+-500159209|-238516.68|ethan steinbeck
+1667577559|-3219150.74|katie white
+1625585280|3470647.28|oscar xylophone
+601018024|4854494.73|david ichabod
+-918780892|-3818373.29|irene van buren
+-1022486818|-184111.46|sarah king
+471621125|437272.39|yuri robinson
+-318437119|1922810.55|zach johnson
+-1182784661|-1749643.71|david nixon
+542831928|3993707.67|wendy miller
+188489461|1841907.17|david nixon
+862815692|-1656673.71|yuri white
+2087385784|4183113.31|alice garcia
+-1683845247|-3627229.19|oscar underhill
+-1564533592|1489756.67|irene davidson
+1320765631|-4166367.92|luke ichabod
+1050851416|-3158465.31|holly brown
+-285065500|4504518.77|fred young
+1432329511|-3210877.4|quinn white
+-759518224|3715469.53|mike ichabod
+1964296204|1186184.25|victor polk
+710310034|-4455686.56|zach quirinius
+-2113513780|2145348.53|luke laertes
+1037339998|4075618.96|zach nixon
+-685492663|-3818601.18|sarah johnson
+619899752|1353468.22|victor hernandez
+814077977|2969131.31|rachel davidson
+1618126798|-4482426.79|fred white
+395572149|-2386619.24|tom davidson
+699541212|2408810.19|wendy hernandez
+-1932204583|-2705745.86|ulysses underhill
+-1444404538|-4490282.22|bob quirinius
+-132500250|193259.52|gabriella hernandez
+1117386537|1845474.23|yuri garcia
+-696439947|4210376.09|alice garcia
+1073784286|-3625772.85|wendy laertes
+-137237933|-2992412.32|mike hernandez
+1231217339|-3994640.41|quinn xylophone
+143776150|3273063.09|luke zipper
+-1274053805|-1602795.26|gabriella garcia
+-415003491|-3726541.71|ulysses garcia
+787379420|4379525.31|rachel white
+-258300|-4523528.91|quinn hernandez
+93556166|-709245.9|luke ovid
+1304344835|596710.62|alice nixon
+1583432095|3343154.31|sarah robinson
+-1602148427|953445.3|rachel allen
+920538454|1061437.57|fred robinson
+-1350936029|2880856.21|quinn johnson
+2120929036|-794283.81|calvin davidson
+249122359|-1538516.22|jessica robinson
+382650338|-659528.62|xavier miller
+1386830061|-2228513.28|zach king
+1300680588|-4950502.18|gabriella steinbeck
+-977605819|-2952017.64|nick underhill
+-1428062056|2356288.12|xavier brown
+1067848172|4344845.94|mike miller
+-906995304|-3086612.36|xavier zipper
+791269283|-3339866.54|wendy king
+1104204480|-4729214.46|tom johnson
+-2050553159|-2322460.02|xavier garcia
+215550107|996086.8|luke nixon
+-1193331651|-4561751.85|holly ichabod
+537455566|3691438.76|priscilla johnson
+-1482972233|-3796650.97|calvin falkner
+796603183|-1302083.47|xavier carson
+-1534989184|1039310.0|quinn quirinius
+-1350195035|-1709002.0|tom thompson
+-831012333|2327869.89|oscar johnson
+-1727028615|-1919011.53|sarah king
+1222928117|-1417793.48|victor underhill
+1938809384|936991.28|nick king
+-1701204480|3806089.3|luke hernandez
+1137473243|2385343.26|katie hernandez
+584603687|1158019.75|gabriella ellison
+-780471033|4220716.08|bob falkner
+588620076|-2215459.36|zach laertes
+-702798363|869132.73|wendy hernandez
+413019054|747447.07|wendy hernandez
+825921734|-1710397.31|ethan johnson
+-884031094|2279760.77|gabriella van buren
+24165749|-2186835.96|david king
+885561242|-1258931.21|ethan xylophone
+2125208081|-3718721.13|luke robinson
+-1017281476|-2961583.91|priscilla nixon
+45668402|-992970.42|luke hernandez
+-86315511|-56266.86|holly xylophone
+-1375851456|-2870372.18|tom ovid
+1503114459|-2749463.04|victor miller
+-1630998739|4472256.45|fred miller
+1261635838|1307496.57|nick xylophone
+1576797732|-896376.69|priscilla xylophone
+901684470|-3766636.25|quinn king
+-1337658163|3227812.74|irene allen
+-1422529398|3992789.22|ulysses johnson
+-513151460|1033514.41|tom quirinius
+1717831617|2955696.55|calvin steinbeck
+-1443142398|3309046.98|xavier carson
+611875609|2487976.25|bob allen
+5429586|4531.4|zach steinbeck
+253697564|4836429.29|priscilla miller
+292828845|4350601.38|david polk
+-671360996|-1824791.73|ethan allen
+1382474328|3427961.88|oscar nixon
+-160540982|-2450031.76|oscar nixon
+1458788940|3268670.24|jessica underhill
+-1009822503|3136101.35|alice carson
+-1483384091|1249215.05|irene nixon
+484701933|248061.26|mike young
+-342380057|-3046437.84|gabriella nixon
+398459788|-2666187.71|quinn laertes
+321160423|-3117879.33|quinn king
+-1732845881|4523526.23|xavier young
+-1865109961|1529009.91|tom hernandez
+862997706|3246681.04|david ichabod
+-1427357995|-3128289.19|oscar davidson
+-783704401|-1823456.44|katie ellison
+-2021262052|1362944.51|priscilla laertes
+58729230|4189752.83|holly steinbeck
+1195913005|2731374.52|holly miller
+-907145136|4197395.25|bob allen
+495580479|4432152.08|priscilla nixon
+-963071836|-960130.12|ethan ellison
+-1273939118|-334427.87|fred carson
+-765726680|1868371.63|luke falkner
+-1469535482|4250752.81|oscar falkner
+1148492727|-2057908.72|luke ichabod
+112547205|-3388986.01|gabriella falkner
+-901958848|248352.95|gabriella johnson
+-145564999|3863922.6|alice ovid
+-2103427908|2353160.84|yuri johnson
+-1265111487|-1814137.22|calvin davidson
+1142625575|-4008613.72|calvin allen
+-806022158|-528390.41|victor van buren
+-219454719|2973551.59|david garcia
+-1635478340|-449625.22|gabriella brown
+-1894049227|3139244.79|priscilla van buren
+-310848952|1263820.21|luke ellison
+-1063222385|703973.42|gabriella steinbeck
+1835569847|19470.25|katie underhill
+-694451715|4610057.25|victor steinbeck
+-254382274|-3205245.33|priscilla garcia
+701197895|-3325408.3|holly falkner
+956272528|-4462219.79|luke brown
+-1491409389|-4808044.65|victor allen
+-1567614615|-1433201.74|ethan ovid
+-76815043|-3956227.95|fred falkner
+-1243445629|-3815726.02|oscar johnson
+-919666433|-2727613.26|bob allen
+558841056|-3099984.45|rachel robinson
+815706703|-1975107.54|rachel quirinius
+-726164420|4913893.04|katie ellison
+1239603583|839107.69|calvin allen
+468388072|2184988.17|mike falkner
+399644425|-1679647.09|nick garcia
+1532796833|2308718.53|bob laertes
+-203108860|2745481.17|nick polk
+-380109087|-3593024.76|quinn polk
+339784381|4635816.7|yuri nixon
+705511907|-1414153.07|victor allen
+1790524518|326890.81|sarah quirinius
+1260762731|2631598.64|quinn steinbeck
+1198537437|-3501859.62|zach steinbeck
+1962496454|-4778142.12|priscilla nixon
+-2131836437|-3608082.2|calvin ellison
+1757496028|-3205545.77|gabriella robinson
+-191913572|-4540656.63|luke nixon
+-1380022743|876794.05|ethan garcia
+2018856823|-4184756.99|david underhill
+1206904293|-1429135.03|ulysses polk
+-259356989|4657037.77|david laertes
+1661248472|-1250047.97|quinn allen
+1337560595|4571610.55|calvin laertes
+-887161981|-4157186.18|alice young
+-1279389993|-691564.34|zach polk
+1179781786|-1164729.5|fred allen
+-2132652124|-4567052.49|holly polk
+1434143522|-52936.3|yuri ichabod
+-944931732|4198579.66|tom allen
+1683092853|-130786.53|priscilla ichabod
+63309795|-2216041.97|irene van buren
+1350773271|-305521.14|calvin steinbeck
+304293826|-4289245.59|calvin quirinius
+-1988822464|3026047.18|ethan brown
+-1251833436|3394024.57|ethan ellison
+1278483264|402248.45|zach carson
+-2112573270|-4966163.29|sarah steinbeck
+-1219521429|-4094684.04|mike young
+-2121169852|3532634.37|alice polk
+-1689397258|-2221531.59|rachel underhill
+-1750628716|4432081.73|irene nixon
+1839318799|-791100.99|luke ellison
+-1008697999|2768132.41|jessica falkner
+-1784025549|4436503.03|david falkner
+974463734|-1089273.41|oscar zipper
+-1614405060|-4801205.69|david white
+337330975|100461.91|oscar laertes
+-191340537|1682613.66|quinn steinbeck
+-97742363|3257772.64|ethan xylophone
+1269151609|1883991.47|fred thompson
+874254793|1921517.5|gabriella johnson
+-1781600194|-303850.72|jessica king
+-922147603|-3240911.86|irene ellison
+1702480288|-2428521.01|victor king
+-1512772886|3564382.93|katie laertes
+-771618739|2582532.61|gabriella king
+-158569421|310005.77|fred van buren
+-757382178|-557651.36|ulysses ichabod
+-559818828|-3993616.98|david nixon
+569598844|-605513.22|irene zipper
+-58994226|-909921.31|holly brown
+1931691011|-2967374.56|ethan ovid
+1611404309|2460625.46|fred young
+417590106|49322.41|mike hernandez
+-804228255|-2694442.89|wendy allen
+841958418|-3392867.2|oscar robinson
+-1166385677|1175371.99|gabriella ovid
+-2094829217|4176283.14|yuri king
+-1500375556|4082157.7|victor nixon
+678866693|-2545586.87|priscilla young
+-1541741410|-2472263.4|ulysses steinbeck
+1681011162|-2041679.96|priscilla robinson
+-528723917|3723274.31|irene nixon
+793177311|2986231.14|ulysses johnson
+-1713400684|1434190.25|alice hernandez
+1993165861|-3268626.55|irene robinson
+468665982|2019123.19|alice underhill
+-1255263751|2425005.61|bob van buren
+-1444037762|-2089765.09|bob quirinius
+-208626679|809173.23|ulysses young
+-787769899|-1147618.43|irene underhill
+878484868|-3121089.11|ethan ovid
+44449892|-3780777.91|xavier falkner
+-1378508696|-3855631.46|tom king
+-1147975547|-2783629.77|oscar laertes
+1850635108|3153377.71|nick falkner
+531360822|98211.26|alice van buren
+-480224460|-3957146.11|bob white
+989851987|-867801.07|gabriella miller
+-2069267150|989069.65|fred white
+457801336|898319.86|mike robinson
+1870681743|2336612.17|rachel ellison
+-1620031269|-520255.87|alice nixon
+-2112168314|3153632.42|holly laertes
+-292451762|1708285.96|gabriella robinson
+-556975294|4904137.08|gabriella ellison
+-1935808223|2070812.35|mike ellison
+-965613428|1394873.44|holly brown
+-1446619225|2441776.76|xavier underhill
+1121166427|4013167.85|gabriella polk
+-1900674185|-3512936.03|quinn falkner
+-1167083317|-4522455.02|irene young
+-918010677|-1021091.89|jessica allen
+-1507845078|-3951541.12|holly robinson
+632695954|-4655315.5|nick hernandez
+-385175762|3009340.58|ulysses robinson
+1572168713|-1503179.88|rachel ellison
+-698651042|-3728243.2|luke nixon
+-1122985232|-3734404.64|victor brown
+544737109|-2507412.25|quinn brown
+1615679932|4558060.89|mike allen
+-2065908870|2429957.68|calvin thompson
+-1390840988|433148.97|mike underhill
+476913954|3838695.07|oscar falkner
+324042367|22790.93|sarah allen
+-881657475|-3537791.09|victor robinson
+-866582526|-4840545.77|gabriella king
+-806009728|-381295.17|rachel nixon
+154298802|804015.15|calvin ovid
+745590224|3032893.36|ulysses miller
+-1500189391|4331184.02|nick robinson
+389016686|-3394921.06|jessica steinbeck
+1767016492|-4494496.99|mike quirinius
+-1887769485|-995870.5|zach falkner
+-394991348|1339126.83|priscilla thompson
+462349651|-1959378.9|oscar ovid
+1304941377|3399851.3|nick white
+1501194339|-3278469.86|sarah polk
+1330987259|510814.95|katie hernandez
+1190840964|-252521.58|ethan brown
+-404800521|-1431896.18|jessica garcia
+178364750|1677124.94|victor zipper
+-1352377964|-2999005.89|victor allen
+1323282192|-2796513.94|gabriella polk
+947748483|600047.96|mike ichabod
+854309498|-4116041.09|gabriella ellison
+-1127342932|1365947.28|xavier young
+-1093993008|449484.65|wendy brown
+316740105|532086.62|holly ichabod
+-1591490830|-2607928.77|yuri ichabod
+-373638720|3937951.67|calvin carson
+-566829909|-4121831.78|gabriella quirinius
+1267282067|4882513.82|nick nixon
+1938953969|2633225.91|ulysses ichabod
+679761032|-1122613.74|oscar falkner
+-979460308|164775.05|rachel hernandez
+378221394|-4844067.79|xavier quirinius
+-785689964|-3366450.06|victor underhill
+-503218724|-2867080.1|fred quirinius
+-1354558391|748531.79|quinn underhill
+-526401771|-3827982.76|victor polk
+667307450|104524.98|rachel laertes
+962106049|-1825748.99|alice zipper
+-816006292|-1879032.68|tom garcia
+1223523590|4390186.17|david white
+-1666006722|-3367283.4|zach ovid
+1807900120|-564758.32|bob king
+1495084229|3826542.65|rachel quirinius
+927947690|-868953.66|david white
+441210200|-4149898.55|david white
+-114256150|44945.06|oscar white
+1762672851|479763.46|ulysses thompson
+2143851546|1536606.11|irene xylophone
+-334321328|3886507.43|priscilla polk
+1698110665|-2774331.54|holly young
+-982990030|4542066.48|victor king
+-1203179649|1281710.53|victor allen
+-190299036|3483906.07|irene brown
+-65206924|3265297.99|wendy quirinius
+952876190|2216066.99|ulysses davidson
+907020490|1685341.14|wendy young
+723491834|-3116769.32|ethan van buren
+-1327701913|2258838.5|wendy underhill
+2139136079|4082737.54|oscar zipper
+-333019459|3927372.2|quinn ellison
+-1737581814|2565928.32|bob nixon
+207210300|-4136553.98|oscar johnson
+-1597249416|-4648978.01|mike quirinius
+-1329034730|-1272065.75|fred allen
+-1879157051|-4173770.3|calvin ichabod
+1315510411|-1407145.84|gabriella ellison
+971304470|-1524708.02|oscar laertes
+1417464854|3985275.86|priscilla van buren
+1347804297|-4758471.06|rachel xylophone
+1959196928|-3869527.43|priscilla van buren
+-592508926|933688.43|gabriella underhill
+-1087362753|3605266.07|xavier thompson
+-1072413430|-4233554.76|priscilla zipper
+1558050488|-4326231.72|victor ellison
+-1520353997|-1996724.55|tom xylophone
+-1102736318|1921519.94|fred davidson
+688879800|1363981.93|xavier falkner
+-1044878622|-687217.08|mike davidson
+-1898506669|3113333.76|nick allen
+1449045322|-251612.99|alice robinson
+-457551497|-1138654.64|yuri ellison
+1773958545|-761507.02|tom ichabod
+-1600103804|3847050.3|nick steinbeck
+-990807690|-1002705.56|fred ovid
+187565438|1728827.4|nick hernandez
+-2043039770|4486640.17|zach thompson
+-1421032590|1779086.27|katie laertes
+617346694|-1093487.56|quinn davidson
+-1231550577|471831.84|gabriella robinson
+-2087587746|732846.99|fred quirinius
+170128539|-1689456.69|jessica allen
+363759342|-4009613.94|bob underhill
+278841432|1636526.65|ethan ichabod
+1646574966|-1882757.08|priscilla falkner
+2092515806|-4434038.47|tom garcia
+1315598848|3730284.19|jessica hernandez
+-1815280343|2490372.44|luke davidson
+274680457|3412161.09|david nixon
+433459522|2665190.38|rachel ovid
+-1833831312|-4551469.71|irene robinson
+159158229|4275302.09|jessica brown
+732502754|-3755821.82|gabriella king
+659350536|-463961.23|xavier ellison
+-834429684|2589729.25|bob miller
+1515109357|-4337250.59|david underhill
+1788323620|-799935.84|gabriella polk
+447369972|-3504949.55|gabriella johnson
+-1097666115|361268.25|ethan brown
+893508891|-4621065.3|zach white
+-1121869604|-3887963.96|rachel king
+-1935111720|1540829.62|ulysses carson
+2126813374|-2844824.79|tom ichabod
+2099282652|722454.72|priscilla polk
+-1045122973|4726775.54|xavier allen
+-1476063648|4468076.48|tom underhill
+-670793185|3904544.03|holly zipper
+-559915677|-2735317.19|alice van buren
+2119616759|379425.9|katie brown
+-887103965|-1844352.58|yuri garcia
+-2044146315|4255709.66|tom miller
+-1605634104|-4099096.32|irene king
+-1382125796|-2248567.56|ulysses carson
+903753703|687046.77|yuri carson
+-964925469|-2382643.36|luke robinson
+-914115721|1694376.28|calvin van buren
+1317768495|2791587.27|wendy young
+-1041603528|-4850949.33|nick hernandez
+-659513779|453599.15|gabriella ellison
+-188763945|-4191721.29|victor quirinius
+1315222092|-3606315.48|david quirinius
+-1512705755|-2810324.25|priscilla garcia
+1754603480|-987641.55|yuri zipper
+1006997815|-2972869.08|irene underhill
+-84748445|3391240.79|mike zipper
+186209624|-1289799.85|mike white
+-1366152024|634411.33|irene hernandez
+-1837482494|3798956.08|luke miller
+-1274542444|3257990.42|jessica allen
+-21930398|1770938.93|priscilla van buren
+-46601002|4202370.57|zach ellison
+-598138597|-4154981.4|holly miller
+-345276207|3969278.99|mike garcia
+-1150792133|2104234.56|ethan polk
+802302895|711412.15|xavier ichabod
+749173400|-3257891.22|priscilla young
+-1538245194|4629754.7|mike allen
+116698141|1091662.27|mike robinson
+1088680755|-3357636.67|holly davidson
+-1493975127|4633419.23|tom johnson
+631300513|521348.76|fred quirinius
+-281097080|-1939464.62|irene white
+879936460|-720862.16|jessica xylophone
+-1332334914|-2387780.52|victor ellison
+-1494317542|1455646.3|david hernandez
+-2037293940|-1984058.19|priscilla white
+1888183374|-458471.61|gabriella robinson
+-512162563|-2814328.93|luke polk
+888008799|-1021669.02|tom brown
+-1149731748|21017.09|gabriella steinbeck
+-878279120|3724869.83|xavier zipper
+-1381236505|-2504122.01|bob allen
+-1972825523|3791143.09|david ovid
+1323185428|-2714024.69|david brown
+-1275610616|-3107352.92|xavier ovid
+674242118|-1822973.02|priscilla johnson
+-341773962|3418175.89|oscar brown
+2143966665|-4614655.33|gabriella allen
+1224044617|2918068.78|bob allen
+-910313633|4480378.46|david quirinius
+1452167294|3806549.93|mike underhill
+-615578669|996402.81|nick zipper
+2040529475|-3603962.3|holly falkner
+-2063042738|2815814.33|priscilla thompson
+1350416056|2343505.66|alice underhill
+-673543594|1087245.88|alice zipper
+-629014700|1146651.88|jessica quirinius
+41011650|2446272.03|xavier ovid
+-613567998|4288441.06|tom xylophone
+1755460884|4871269.13|calvin king
+240401003|-3177057.47|nick quirinius
+-395070488|2630087.31|luke polk
+1637641949|-1576150.92|ethan king
+2137080441|-3779148.71|wendy ichabod
+668750790|-3544674.06|tom miller
+294863494|-1969828.66|david davidson
+-1105984816|-143319.44|quinn hernandez
+-1015400224|-1924763.44|sarah davidson
+860142315|2568293.65|wendy ichabod
+1570437505|-3714049.34|ethan ovid
+-1543305986|353326.08|calvin falkner
+1827154722|-1991738.99|katie xylophone
+756114658|965016.73|david steinbeck
+-142937999|-1044256.14|tom johnson
+243880195|-3162122.18|david xylophone
+-895960982|3206691.89|yuri robinson
+-1523273034|-714625.45|yuri king
+-323647332|-2140875.17|priscilla carson
+-1114173417|2164053.17|priscilla quirinius
+771674930|-3840697.76|tom young
+1910785065|1113763.2|jessica thompson
+1945430042|1549971.76|priscilla davidson
+349070428|4894338.04|mike steinbeck
+-1271817312|-776092.59|bob carson
+1877324477|-3425027.24|sarah xylophone
+1334224458|3194212.97|calvin ellison
+-1790553460|-4537506.02|ulysses miller
+788125950|-3378292.17|oscar underhill
+-513838876|2058273.48|katie davidson
+323507571|-431603.48|xavier nixon
+483902911|-896728.94|nick polk
+-1861493089|-1653568.83|bob thompson
+-1995814530|-3280803.08|alice van buren
+1699272119|-2113479.34|oscar falkner
+-1417426527|3620431.94|nick steinbeck
+261285840|-1145288.71|zach nixon
+-500417637|2711667.41|yuri ichabod
+1783078204|447207.14|katie davidson
+-1681631563|-4785775.3|calvin van buren
+1264990011|-1547602.63|katie laertes
+-2041729422|2405209.67|quinn davidson
+-1265868386|-1990829.67|alice polk
+-1955115774|3954195.7|luke johnson
+1962902884|1081319.58|alice falkner
+-463586914|-3933241.25|quinn polk
+697224845|-1218328.41|xavier white
+315455719|3283412.12|jessica johnson
+2060814973|-4841536.41|irene young
+1379827098|1276705.96|victor hernandez
+1120875981|-3016357.47|david polk
+1534967355|4365220.64|yuri hernandez
+-246108306|-4515709.51|holly thompson
+1705035904|3565558.11|jessica zipper
+-715758511|199551.47|holly van buren

<TRUNCATED>

[4/4] hive git commit: HIVE-14214: ORC Schema Evolution and Predicate Push Down do not work together (no rows returned) (Matt McCline, reviewed by Prasanth Jayachandran/Owen O'Malley)

Posted by mm...@apache.org.
HIVE-14214: ORC Schema Evolution and Predicate Push Down do not work together (no rows returned) (Matt McCline, reviewed by Prasanth Jayachandran/Owen O'Malley)


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

Branch: refs/heads/master
Commit: b28ec7fdd8317b47973c6c8f7cdfe805dc20a806
Parents: 9eb070a
Author: Matt McCline <mm...@hortonworks.com>
Authored: Sun Jul 24 01:23:03 2016 -0700
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Sun Jul 24 01:23:03 2016 -0700

----------------------------------------------------------------------
 data/files/unique_1.txt                         | 5000 ++++++++++++++++++
 data/files/unique_2.txt                         | 5000 ++++++++++++++++++
 .../test/resources/testconfiguration.properties |    4 +
 .../org/apache/orc/impl/RecordReaderImpl.java   |   31 +-
 .../org/apache/orc/impl/SchemaEvolution.java    |  101 +-
 .../apache/orc/impl/TestSchemaEvolution.java    |  107 +
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |  167 +-
 .../hive/ql/io/orc/OrcNewInputFormat.java       |    2 +-
 .../hive/ql/io/orc/TestInputOutputFormat.java   |   16 +-
 .../hive/ql/io/orc/TestOrcSplitElimination.java |    3 +
 .../clientpositive/orc_ppd_schema_evol_1a.q     |   42 +
 .../clientpositive/orc_ppd_schema_evol_1b.q     |   60 +
 .../clientpositive/orc_ppd_schema_evol_2a.q     |   43 +
 .../clientpositive/orc_ppd_schema_evol_2b.q     |   62 +
 .../clientpositive/orc_ppd_schema_evol_1a.q.out |   70 +
 .../clientpositive/orc_ppd_schema_evol_1b.q.out |  124 +
 .../clientpositive/orc_ppd_schema_evol_2a.q.out |   70 +
 .../clientpositive/orc_ppd_schema_evol_2b.q.out |  124 +
 .../tez/orc_ppd_schema_evol_1a.q.out            |   70 +
 .../tez/orc_ppd_schema_evol_1b.q.out            |  124 +
 .../tez/orc_ppd_schema_evol_2a.q.out            |   70 +
 .../tez/orc_ppd_schema_evol_2b.q.out            |  124 +
 22 files changed, 11333 insertions(+), 81 deletions(-)
----------------------------------------------------------------------



[2/4] hive git commit: HIVE-14214: ORC Schema Evolution and Predicate Push Down do not work together (no rows returned) (Matt McCline, reviewed by Prasanth Jayachandran/Owen O'Malley)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/b28ec7fd/data/files/unique_2.txt
----------------------------------------------------------------------
diff --git a/data/files/unique_2.txt b/data/files/unique_2.txt
new file mode 100644
index 0000000..f56d669
--- /dev/null
+++ b/data/files/unique_2.txt
@@ -0,0 +1,5000 @@
+-530398323|3148236.99|ethan laertes
+1413113206|-2202479.12|david brown
+-791175596|-4537760.31|irene allen
+-863449193|-3335467.65|sarah garcia
+657672399|1561192.1|oscar ovid
+1585776606|3003821.68|zach quirinius
+74855397|2799776.53|yuri quirinius
+-611608318|-757864.7|gabriella white
+-390002017|-2955212.58|calvin king
+-1738550115|2886475.2|jessica thompson
+-1130695756|-3885663.52|yuri young
+1981152190|2377002.46|fred thompson
+-1385751468|-3571023.23|sarah nixon
+-680481043|-1436863.75|xavier underhill
+-1834919444|-2163504.3|oscar ovid
+-1820717770|-2535283.55|mike ellison
+-719144984|3406466.15|fred robinson
+-1313204690|-1126273.06|quinn allen
+928893846|-3522460.34|ethan steinbeck
+-1267350292|-4385782.69|wendy van buren
+356187071|1457729.0|jessica falkner
+1457488479|1512688.71|rachel ovid
+-251307384|148477.27|victor young
+357960601|3173180.99|ethan ichabod
+1894748086|2300695.0|ethan nixon
+1371819228|-2483333.39|ethan garcia
+-1851298170|1609219.48|nick underhill
+262016140|4196122.49|jessica quirinius
+-335577745|-3878366.66|gabriella young
+-1284300453|1185746.64|rachel johnson
+1220853691|1225388.32|ethan brown
+-975739584|-542374.88|calvin carson
+-2113353772|-4749716.75|victor quirinius
+1513108256|-4253461.95|ulysses zipper
+175918688|-2674810.67|wendy ellison
+194645814|3141302.76|katie polk
+-826503696|2264222.64|ethan ichabod
+688435847|-4100066.3|rachel falkner
+564151202|-902682.78|oscar van buren
+2103517212|1964748.72|holly falkner
+-1075133194|-120426.31|ethan miller
+1902001844|4071621.45|bob quirinius
+-720771112|-652888.98|zach brown
+2040773663|1265014.46|bob robinson
+1723585927|3885939.61|bob xylophone
+-518697068|1654435.18|irene young
+888478182|720022.52|zach garcia
+1428674187|-4529952.07|yuri robinson
+-665999285|2135213.6|xavier laertes
+-1854811113|-2027745.33|rachel zipper
+944305139|3090549.92|tom carson
+-1042375515|-1110292.31|luke johnson
+580185516|4862616.16|oscar nixon
+1686995118|2997966.46|fred ellison
+1153206601|3976509.09|priscilla robinson
+403744457|-2685091.62|calvin allen
+-803558155|1913765.12|ulysses allen
+-2047899126|-865688.26|katie miller
+888924556|1442007.53|luke young
+-2022552633|2423348.1|holly xylophone
+-1087281524|-4372719.89|wendy hernandez
+1717946382|-1553650.24|mike polk
+2123552031|-3039358.07|irene thompson
+1193718392|3332073.1|calvin king
+785143176|-1124364.95|gabriella king
+-267158249|4666585.23|nick carson
+2075119315|-2820100.3|xavier carson
+-1560117848|2156551.78|yuri falkner
+-1619092350|3138714.72|bob miller
+1979333995|846122.45|nick nixon
+1007384272|108898.87|irene falkner
+1472540191|447412.36|mike brown
+1502570896|3359569.46|yuri quirinius
+1179497221|-316152.55|jessica king
+-240821559|-4311410.54|zach zipper
+-2117953776|-1337250.73|priscilla quirinius
+885218723|3938711.64|katie underhill
+15415682|2035678.35|ulysses ovid
+-757553318|-665746.22|sarah brown
+2144600230|2147875.87|yuri polk
+1387971336|2634224.63|oscar van buren
+1469222502|1811193.33|jessica ovid
+-1865914685|2296223.99|nick ovid
+-2120035091|656298.34|katie ellison
+1422876472|-2234025.76|sarah miller
+-1673719402|1048021.73|ethan garcia
+-1601423158|-4408427.51|oscar ichabod
+179630813|-1555039.19|oscar garcia
+-1183585308|-4766909.04|quinn van buren
+661475853|-4723403.53|priscilla king
+337358828|-3063734.32|fred brown
+-447413300|3071668.0|katie steinbeck
+1316651835|-951856.93|calvin miller
+-725032216|-4070708.4|jessica garcia
+-1896366137|-1733205.31|jessica ichabod
+-809537803|839982.27|david hernandez
+1687587717|-4183687.59|bob falkner
+2080552079|-2692562.76|xavier underhill
+-1623267131|-3110751.63|holly van buren
+1965475682|-1527480.61|david king
+-668340770|-4365568.36|priscilla zipper
+-951292349|1014792.15|sarah zipper
+-143520503|-3567435.01|xavier allen
+-110571517|-4888114.08|nick zipper
+1261741128|-112290.43|xavier polk
+863077320|533090.65|holly laertes
+1693784732|-3894570.2|mike johnson
+-739282424|-2317287.54|xavier falkner
+-1756304640|-1640124.48|wendy miller
+2024211315|4209343.46|bob king
+210237147|3350867.69|gabriella polk
+-629718654|500295.7|mike young
+-1673158406|1767580.1|bob thompson
+1782536279|2019489.35|yuri ellison
+1037129499|-2959347.46|gabriella miller
+-108041159|-3248029.72|quinn carson
+-81550406|1448214.25|quinn quirinius
+1460763850|2461344.28|nick johnson
+-891391328|-4371390.18|oscar polk
+350039758|3045942.61|luke king
+-296078757|481615.89|rachel laertes
+-1463854817|2422905.47|luke allen
+628079036|-1943134.07|yuri miller
+650062868|-2972863.88|victor zipper
+2012324824|2363940.32|xavier xylophone
+1969179293|-1799982.62|gabriella steinbeck
+-481558966|-855011.93|nick brown
+-736410964|3477573.65|ethan ellison
+1332411552|-3326458.14|bob quirinius
+-1314844986|-2450976.99|rachel ellison
+-1310672564|-3754597.71|ethan davidson
+2866398|3195661.41|sarah underhill
+-606195877|515065.79|xavier king
+-241791291|-4977114.37|sarah quirinius
+426727905|-730134.39|zach steinbeck
+1397159614|-3901106.82|david brown
+-1273490186|-716152.82|wendy brown
+672483126|-660299.15|rachel quirinius
+-835925650|2025241.34|david brown
+1360975487|701165.88|ethan hernandez
+2028759026|-2813327.63|priscilla king
+-1871886306|-2624394.63|quinn van buren
+860459268|2980275.49|katie steinbeck
+1427354386|4631042.12|alice xylophone
+-722514186|3559244.85|quinn underhill
+406260000|3983654.18|jessica falkner
+1310084119|-1949862.76|nick king
+1076921602|-1098385.52|katie davidson
+999637641|4251326.44|irene young
+-411464184|-3641138.39|jessica ovid
+-811510686|-4742441.99|mike johnson
+-1703117420|3905573.37|zach white
+-275817933|1967261.86|fred king
+2024183670|1458610.65|rachel allen
+1451217684|827623.53|bob underhill
+-2101321703|-251141.65|luke young
+2019380409|-3444043.65|oscar falkner
+2015550440|2127107.74|nick quirinius
+1245029183|1610529.91|zach laertes
+-1024706155|2679835.24|zach van buren
+549302194|-3212372.99|gabriella van buren
+8684783|3361325.53|rachel white
+320647553|-990448.39|ulysses young
+1446269588|3613990.85|oscar thompson
+1176519249|2425804.57|fred hernandez
+-576557910|2611510.75|tom miller
+1040565491|2720474.05|jessica robinson
+1459779403|2661127.41|ulysses hernandez
+600133486|-2463110.08|alice king
+691722117|4368168.17|alice zipper
+-950936237|-2442040.78|sarah laertes
+-1204096529|4166172.11|nick hernandez
+-1643613481|-1545416.72|irene young
+-1465866696|-1495459.3|jessica falkner
+926192236|-3958371.25|fred hernandez
+223531857|2077947.59|katie hernandez
+850677203|-2045030.45|bob carson
+-406701062|-2907179.59|irene ellison
+-177397554|1921705.83|yuri steinbeck
+-2121308733|2933279.43|zach nixon
+896005886|-1334022.54|bob nixon
+-1287809788|-4413778.3|zach zipper
+-1085468758|2716246.55|fred ovid
+-1631787482|-1173900.5|katie underhill
+-1974587068|-3957903.11|nick ovid
+-1789834932|468955.47|rachel hernandez
+479763468|978452.76|luke thompson
+-1459822767|1330943.18|oscar miller
+417969717|4235154.9|nick young
+-1601013886|-2447860.44|luke laertes
+535713088|-3623817.06|oscar ichabod
+-1103445079|-1655613.28|mike quirinius
+-1278664812|-3418118.33|jessica young
+-1549544379|-1915405.8|zach carson
+1798346365|1803206.1|gabriella brown
+245342460|3542231.7|bob quirinius
+2077244442|-1750041.23|yuri carson
+-621127253|-4479120.57|bob van buren
+19188183|3303366.08|david nixon
+-29095263|4939069.22|sarah thompson
+766237580|-1279001.31|gabriella davidson
+357061359|-4868836.39|gabriella ovid
+-1201561499|3318781.44|zach xylophone
+-1046374962|-2336548.65|ulysses johnson
+-247964158|469973.41|jessica falkner
+-1129332612|4353644.91|mike van buren
+1777773484|1009389.17|oscar miller
+-1675393992|2278213.34|nick quirinius
+1549963885|1498255.04|jessica robinson
+1184976921|804110.51|ulysses garcia
+624231509|-434810.17|david thompson
+-1125091087|-4997768.79|tom allen
+1620422769|-292330.01|rachel van buren
+-1634900618|811388.25|fred young
+-1561728496|4569857.93|alice van buren
+1339904892|-783481.61|ulysses allen
+-765429402|-2274434.95|mike davidson
+1619586775|-2896748.29|tom ellison
+-361658178|-3305914.78|wendy white
+-1716810443|-1677735.12|luke davidson
+2044644517|-3690645.85|bob hernandez
+1358955038|4404859.29|tom polk
+193548004|-2987984.06|gabriella ovid
+1568912606|988156.73|ulysses steinbeck
+-18597092|1320625.44|nick van buren
+-775651841|-4970130.2|tom xylophone
+368476504|851054.04|bob young
+-1337531250|-825214.75|zach king
+958359053|4254653.54|katie quirinius
+58141565|-1439440.95|alice miller
+489617444|1704561.64|priscilla white
+991448502|3119616.92|wendy johnson
+699741480|-4924151.17|tom king
+1756765099|-4836107.97|ethan zipper
+1333211321|-3147331.64|xavier falkner
+-1818085484|2240050.89|zach quirinius
+744894412|1621798.48|xavier underhill
+-689750623|-2569338.11|ulysses young
+-1768283812|1036460.96|fred steinbeck
+743967676|-3833507.14|oscar allen
+1366263193|4920379.22|nick allen
+452618426|1528317.88|jessica hernandez
+700217800|-3135.38|zach thompson
+-1148105469|822826.02|ethan laertes
+-1438346694|-418035.02|zach young
+1208899351|2317548.33|priscilla quirinius
+622214441|-3303049.42|bob johnson
+1007654031|3567615.01|zach hernandez
+154462320|2991778.91|ethan falkner
+887114833|-4490611.26|fred garcia
+181592381|-4535649.47|fred white
+-1768689348|2304039.22|priscilla ellison
+-95014814|3689464.63|nick robinson
+1188979832|1263668.77|fred van buren
+-784769339|2581817.24|calvin polk
+-120175312|2536872.62|holly xylophone
+-774347486|-4718758.15|priscilla underhill
+-1897347243|399821.8|bob king
+1389027113|1575383.58|priscilla thompson
+347831744|-4553491.1|zach young
+-1668924030|-2974418.22|ulysses brown
+802044891|-4124963.55|irene laertes
+-25169037|1751429.03|sarah allen
+1364050308|-1022470.6|xavier steinbeck
+-1579496423|250953.91|wendy ovid
+-2084775951|-3131242.24|wendy miller
+502366316|4660404.73|david falkner
+268169401|2985827.01|ethan davidson
+1594401423|2732373.84|yuri ichabod
+-181680033|901726.05|jessica steinbeck
+-297022446|3953885.6|yuri robinson
+-633072680|1905787.88|yuri king
+1590559792|2644486.38|bob brown
+-1253610220|-4664099.42|irene white
+-861448130|4484652.1|ulysses miller
+-1067019148|2908195.28|zach allen
+-834866255|3638766.25|sarah nixon
+-1779874399|4491332.65|wendy laertes
+1879864977|2044.35|holly hernandez
+-1353294761|809811.02|alice white
+-235591620|1720874.56|rachel king
+1004628464|2838655.31|katie brown
+1614528935|36019.67|rachel brown
+-1391708450|225211.64|david underhill
+-1125042751|-965210.61|mike allen
+-2012247223|361423.63|gabriella laertes
+-1181292680|3756070.82|alice nixon
+-51244908|747362.65|jessica thompson
+1631024255|-4659701.08|priscilla johnson
+-1631733377|-470101.64|rachel allen
+-1183682295|-3957933.34|jessica quirinius
+1149563148|-2098692.54|oscar white
+783971280|3190820.04|zach ichabod
+-48273282|-2595405.62|victor van buren
+1274180442|1613551.62|xavier zipper
+416467715|4799681.54|fred laertes
+-323222474|714051.9|calvin nixon
+-251432455|3430342.54|sarah falkner
+-627485293|826224.15|nick laertes
+2052816566|872739.46|irene young
+-782687070|-3052794.01|mike brown
+-554692754|-2555183.08|david steinbeck
+1243682919|3247099.22|luke ichabod
+-598037886|1751156.91|rachel thompson
+-1597743349|-796260.94|irene nixon
+-1555822499|1483287.44|gabriella falkner
+2085931724|-3955114.15|ulysses carson
+1893344697|817743.14|calvin falkner
+1834967084|2615667.72|ulysses king
+332959944|-1993651.73|katie quirinius
+-1062660128|-4145231.59|priscilla davidson
+-1167900213|-1446173.56|ethan zipper
+312202911|-3685570.94|jessica thompson
+87790574|320873.81|bob polk
+-1987569135|-1533789.33|victor ichabod
+1492594067|1495657.91|wendy nixon
+1234101532|3537386.91|bob robinson
+20080452|4036544.07|mike falkner
+233168789|-2598505.1|fred hernandez
+-932899070|2262670.78|holly ichabod
+514962443|-1026267.69|sarah hernandez
+403251636|1449788.66|victor ichabod
+1946176525|3258105.38|yuri carson
+1281287914|-4931463.04|xavier young
+1738862160|2933706.99|rachel johnson
+1669534489|541500.5|luke garcia
+-463060108|2230157.02|tom ichabod
+-1458326740|4692132.0|katie johnson
+1966355798|323553.95|katie ichabod
+760027130|2800696.1|luke ichabod
+-926463418|-2753003.85|bob xylophone
+-136756476|-1148041.65|mike carson
+-997599139|2163886.65|gabriella davidson
+1809499525|-2951473.78|yuri johnson
+1105069812|-4991247.61|holly carson
+-187940430|-4953763.81|mike robinson
+-1849533697|-2426467.57|david ovid
+-472143410|1759496.73|oscar young
+-129675121|-709361.29|jessica young
+1988688717|-587543.42|victor zipper
+1629132550|-3829637.0|mike falkner
+-1573512120|4702023.79|ethan steinbeck
+779138729|-3744428.24|alice carson
+1651818467|1214153.72|jessica johnson
+1644856692|974940.45|victor steinbeck
+1592875650|3909847.05|sarah garcia
+-342446445|3277356.92|wendy ellison
+-1061009194|-3162802.67|oscar brown
+-872285492|3745496.3|priscilla allen
+1425075494|-2370561.76|irene polk
+693883319|4647142.99|alice davidson
+-1851147150|-700016.09|priscilla underhill
+1584128670|1671090.88|jessica johnson
+420974895|-3234802.15|yuri hernandez
+-1111762606|-2679259.12|alice quirinius
+-1955124270|-3652779.06|gabriella falkner
+811058778|2294784.27|jessica polk
+1570251113|1981508.78|sarah davidson
+-783924394|-2035891.18|oscar ovid
+-312957196|3270003.68|wendy polk
+787524437|3065055.59|sarah laertes
+1881619502|-1245060.83|wendy king
+-997602147|462975.68|tom miller
+1735869208|-2791216.41|wendy steinbeck
+-1879337961|1000049.31|sarah carson
+-1146576545|-1243862.52|xavier ovid
+-1749341103|-4131907.53|priscilla white
+-1406881951|2352573.19|katie underhill
+641151206|4417771.09|ethan quirinius
+-65567584|653067.31|irene polk
+41991322|-4368387.04|tom zipper
+1510028198|118027.94|alice davidson
+-874411113|2702117.65|nick steinbeck
+-167161898|157767.06|irene johnson
+680918153|3727996.12|oscar falkner
+2050241944|3015344.39|bob van buren
+-484459885|-3350629.8|gabriella robinson
+-293170715|-500882.43|rachel xylophone
+535762756|4713737.89|rachel miller
+174301210|3892418.43|yuri young
+1875306279|3426025.15|luke polk
+1548621220|3824238.05|quinn miller
+-1620714379|-2572932.51|rachel miller
+-700135202|4971371.61|nick hernandez
+1230320400|-1236155.16|gabriella white
+489315621|3859934.41|david johnson
+654759403|-1581875.26|david ovid
+-89411525|398079.32|quinn laertes
+-503035577|-737200.02|luke carson
+554827044|4770673.53|nick underhill
+-1374787943|4482460.75|fred hernandez
+-1574717612|-3269823.48|priscilla falkner
+-1207523831|-1048089.51|victor ellison
+-174615347|4017179.01|luke king
+1942691503|90115.06|david johnson
+329197676|3175376.86|priscilla steinbeck
+-1686904279|3300388.23|ulysses allen
+-554279503|1355018.09|yuri ovid
+2032906516|-3103091.6|rachel xylophone
+-722915961|-4288227.35|bob robinson
+-177762919|-2750791.51|bob davidson
+1829539848|1339302.28|luke garcia
+-1778633207|-1174407.04|luke ovid
+377491772|195790.01|irene johnson
+1721853236|462303.78|victor robinson
+1779767083|-1037564.55|ethan van buren
+671065878|-4062488.58|ethan laertes
+-1550588445|881108.16|sarah white
+-1395855406|-3363251.8|calvin king
+187585100|-2291960.63|yuri ichabod
+1214758975|3099572.44|sarah van buren
+1230499547|4591317.13|oscar white
+1488572037|-4533900.21|ulysses brown
+-671883911|3435847.53|gabriella hernandez
+-152128923|-1500454.27|oscar underhill
+-756848581|2237338.39|ethan zipper
+-1669510087|-3332243.67|tom laertes
+1927651023|-1581689.91|xavier young
+507728627|1681552.86|calvin xylophone
+-322402147|-956630.37|nick laertes
+1106311276|-1345225.45|irene davidson
+1749455873|-4452524.18|oscar laertes
+80852181|-1136426.58|wendy robinson
+999217476|1306792.91|irene thompson
+-1484170935|-1910133.35|fred young
+1221205004|456737.45|ulysses zipper
+477582999|3853938.61|jessica king
+-1130348423|-198693.99|yuri xylophone
+-1236074261|-3817259.99|priscilla thompson
+1896790323|-1988676.08|jessica brown
+346247639|-1507961.95|rachel davidson
+-667526629|40455.01|nick robinson
+-280784402|-1496151.26|yuri white
+180979521|2487825.12|quinn robinson
+570439664|-4118539.75|bob xylophone
+-424674678|-4978536.47|fred ellison
+-1959297223|4953997.22|david brown
+-1595547136|-2320881.51|luke young
+374245353|36913.72|rachel miller
+-89327686|2206224.56|luke carson
+-550726903|-2140159.73|priscilla thompson
+1492711410|3238162.22|katie robinson
+-942937701|-4701325.45|fred falkner
+-2053073775|-724252.56|quinn thompson
+-1194555071|-1186321.89|victor thompson
+-763280038|-2357696.33|mike ovid
+2057607711|-57131.44|calvin xylophone
+-2072434044|1493188.4|ulysses zipper
+-1251325711|-1460687.47|wendy thompson
+1404973925|1267852.28|luke steinbeck
+1035580802|-1110738.15|nick thompson
+-943329935|1674540.88|irene thompson
+-93353692|785129.15|sarah quirinius
+1287411642|-3019737.52|luke steinbeck
+532695459|-3069206.52|tom young
+883549630|4109575.55|quinn ovid
+-1994396434|-2094234.84|katie underhill
+264179750|1354691.38|tom laertes
+-1616748948|4224979.55|fred miller
+2083355827|-918098.02|irene polk
+1271521972|-3376772.35|yuri young
+-1564969311|-149983.59|fred hernandez
+73162545|1234438.04|alice carson
+1264465684|658027.48|ulysses white
+524079805|1529410.99|rachel davidson
+-1270552188|890448.38|david steinbeck
+1250857323|-1686228.78|yuri nixon
+-899253756|-1628282.66|nick miller
+2078763257|-2379362.76|luke ichabod
+2011145226|505233.14|irene polk
+-370965351|1738285.7|nick steinbeck
+-375556587|2511267.05|sarah carson
+93849105|2800878.08|luke underhill
+-1526586609|3799928.41|katie nixon
+-545151122|275671.58|alice xylophone
+1539413168|2581684.61|yuri johnson
+-2101566544|-4542775.17|zach thompson
+869814783|-3337550.25|zach van buren
+-1640010177|-4872308.54|luke miller
+-1302461521|1788202.74|luke garcia
+1802418190|-653778.75|nick ovid
+476560416|-457671.06|irene king
+-562121040|-1978212.08|alice miller
+923155246|-4832300.17|priscilla allen
+282256580|-1076123.34|tom quirinius
+850935302|399997.52|ethan ichabod
+-810724196|2812695.35|wendy davidson
+1205377464|1344957.83|tom van buren
+94076111|-1459698.58|luke nixon
+-671550637|740725.83|nick garcia
+430830659|1655779.56|priscilla ellison
+31158934|4344428.28|zach ovid
+-200266274|1580467.05|calvin young
+906255533|3480192.14|fred miller
+236830138|-2275183.94|nick allen
+1317415533|1947267.47|priscilla hernandez
+-65251104|3487856.26|rachel falkner
+-330630651|4909354.78|yuri ichabod
+-1379375727|-1251102.76|alice robinson
+1164066338|-4281459.16|nick nixon
+1741743568|823748.68|david ellison
+-884222714|2627931.49|victor young
+-152725826|4111519.56|irene van buren
+-1520021524|-2019860.34|rachel underhill
+1706925762|-2898648.31|bob allen
+-2103527463|3609682.06|priscilla thompson
+-1305484847|-1241214.95|irene steinbeck
+-1245735244|-3920699.13|irene allen
+969498153|2444449.31|luke brown
+90967074|-121078.83|david ellison
+2026282328|-834051.95|oscar thompson
+-2123609164|677629.16|zach young
+905944171|-4648030.92|zach miller
+641900086|-3834914.0|sarah davidson
+1884906386|-4331111.76|gabriella johnson
+1452536397|4306889.82|yuri white
+-664459095|543875.05|katie thompson
+1170981021|3628326.15|david ichabod
+-1483400875|-4270114.62|yuri xylophone
+271619627|-1079973.33|wendy johnson
+-1198712634|1404848.89|oscar zipper
+1680509896|4800299.44|nick thompson
+-1485854151|-580549.18|alice hernandez
+-1466016871|-3107937.11|gabriella quirinius
+-493208385|658836.08|bob white
+-350538220|-2534275.07|sarah hernandez
+-1207900660|395825.7|zach xylophone
+876936733|2147718.62|luke garcia
+-992435922|171976.83|mike carson
+-747913218|3745590.91|alice johnson
+777268088|-2117272.71|tom steinbeck
+-687269536|-4468570.87|luke white
+-1402606314|4109726.59|ethan nixon
+-1956155225|-3725179.04|irene zipper
+1967639536|1109402.51|zach falkner
+-741779400|-2119573.92|ulysses ichabod
+-1145657611|-2174135.5|mike quirinius
+696966793|-3543009.13|irene allen
+727995612|2004173.89|yuri robinson
+488498706|2670853.57|katie miller
+1583983879|3940105.84|victor steinbeck
+-410091398|2406443.82|rachel xylophone
+-740873721|2836326.2|ethan thompson
+270450315|-2292515.69|quinn johnson
+537554143|3014441.94|wendy laertes
+-1043725598|2652697.24|sarah thompson
+380181828|-2711975.12|ulysses miller
+1465890630|-2284817.93|zach polk
+-497179099|-2642856.72|sarah thompson
+107142455|-999048.81|alice ellison
+757264230|-991631.39|calvin johnson
+-1814601713|429277.94|mike hernandez
+853263528|-4097654.71|fred ichabod
+-85662782|352122.56|alice quirinius
+-799611294|-4950199.55|mike nixon
+-448260813|1905692.7|quinn robinson
+-1320215119|-3689193.86|fred ellison
+-569650804|2741220.44|calvin robinson
+842523835|300823.77|zach robinson
+-230629348|-718100.69|luke underhill
+1785437309|-2263951.94|quinn nixon
+48826023|-2472126.65|victor nixon
+1120120377|-3210066.91|fred johnson
+-1423572430|3587380.51|priscilla brown
+-740251103|-3234316.52|rachel underhill
+-1569655129|4459997.04|alice robinson
+-1144044087|819264.69|victor thompson
+-634960399|1866551.29|mike ovid
+-484615041|-2609949.27|mike miller
+353895255|3342280.43|tom davidson
+1278841000|4661840.2|rachel thompson
+635204165|-2253614.13|wendy young
+-134072157|-1677143.59|tom johnson
+935247806|3105444.9|katie xylophone
+1340713695|4986306.62|fred ellison
+-1683063761|-3467831.48|jessica davidson
+139138114|2298608.24|mike miller
+-771631272|238926.71|tom davidson
+172726492|314411.14|alice king
+478040720|4293579.49|tom zipper
+1792467603|-859760.23|jessica garcia
+1750209932|-3413677.92|sarah quirinius
+2117550697|-3680522.08|tom garcia
+929063002|-2475815.71|irene hernandez
+-1436911534|-687758.67|gabriella johnson
+910510451|-4604052.32|priscilla van buren
+361947862|3383017.59|gabriella falkner
+-1912200996|-1396479.56|holly underhill
+146707208|-3421663.83|jessica nixon
+1436090827|1973283.24|david underhill
+366172771|1879190.12|calvin brown
+-624074143|3214323.59|nick ichabod
+-2035765335|-3659457.21|oscar johnson
+-928752326|3221707.34|zach xylophone
+-906359570|-415203.25|zach allen
+61281042|-4711293.44|calvin falkner
+-793150390|1664358.9|quinn ichabod
+733375166|-2574611.53|wendy van buren
+-1340524066|3441189.63|irene underhill
+1432204301|-2834244.28|gabriella white
+-316477125|-1847444.57|mike polk
+-463808900|-315219.06|irene brown
+1560082765|-3256902.48|luke davidson
+1920323267|2097239.93|david steinbeck
+-1147445936|-4678803.96|david xylophone
+1744232105|-959023.74|ulysses falkner
+496519803|-1589236.12|ethan johnson
+-1372518252|664892.99|jessica underhill
+-1365118612|-596735.57|priscilla miller
+-1998271277|4319003.62|oscar robinson
+1082318706|-4877379.52|luke polk
+1365684388|-4832267.01|david ellison
+-1027303603|-3651296.71|zach ellison
+40136532|-2400363.67|calvin quirinius
+-345140745|146365.72|oscar davidson
+-1708465144|-2334424.46|victor white
+-698474601|4839511.99|irene king
+-727494888|-3269327.45|priscilla quirinius
+641248688|-3799077.11|irene falkner
+2061476876|-341515.87|jessica miller
+-1521020126|-4626670.81|irene king
+-2096652914|2867096.42|bob white
+2022210458|-1301790.02|tom xylophone
+529103588|-1156665.22|luke falkner
+-535645305|710908.12|quinn quirinius
+1141708522|1358817.93|david quirinius
+-460211539|1433600.91|luke thompson
+-614757768|988654.72|alice king
+-165873944|-4601908.33|luke ichabod
+2074301605|-3222585.0|gabriella ichabod
+-341773880|-1007212.79|tom polk
+983900551|646881.8|luke robinson
+598051582|2671076.9|tom quirinius
+-78033873|-2899213.45|alice thompson
+-2083207454|-310398.41|tom white
+-790809850|1587382.61|nick hernandez
+1335307113|-1811559.46|david zipper
+117044720|1386950.23|mike ichabod
+-874199181|4207130.06|nick quirinius
+-709473357|-2139080.4|bob white
+241817396|3901843.14|ethan ellison
+-992847734|-2785790.24|priscilla carson
+-690270689|-1160242.13|gabriella miller
+2033181178|-3857529.71|calvin white
+921285030|-3479443.65|priscilla miller
+-823561144|3012153.45|nick ovid
+128086436|-2522941.79|ethan quirinius
+-608166899|4258831.54|sarah davidson
+2035522127|2827730.18|quinn van buren
+171887334|-357244.33|zach young
+946415877|3475780.48|nick robinson
+-13408194|4873489.0|alice steinbeck
+966510674|-4702848.61|tom ovid
+-1293123703|418313.66|holly thompson
+-713617159|-4828481.26|zach davidson
+-1661107297|-4574248.64|fred xylophone
+-89000720|2610404.87|jessica king
+-468825355|-1413106.39|zach carson
+1396866701|-3081418.63|zach robinson
+1102170842|-675156.67|bob allen
+761389395|-298145.49|xavier allen
+2026815506|2318957.56|bob xylophone
+365284127|-259470.91|irene garcia
+560918869|771973.15|oscar nixon
+2034647972|4622471.74|zach allen
+1698208934|4752227.1|zach polk
+1080427902|1543943.61|nick laertes
+-260277629|-1417456.38|jessica laertes
+744974150|-4083223.17|quinn garcia
+-106814899|-2224312.0|calvin laertes
+-1417776034|-392879.26|priscilla robinson
+1684443525|-1866386.91|gabriella falkner
+-2050093964|-4702716.19|ethan zipper
+-1896041548|4256779.38|xavier brown
+-1005480454|-2989806.3|luke hernandez
+1993284361|-4335862.79|luke garcia
+-204582149|-531128.83|katie zipper
+-45927906|2355821.49|katie robinson
+-1401321204|-2275927.97|ethan young
+-902750877|4171887.58|wendy nixon
+391494742|136203.04|sarah xylophone
+-739387625|4664944.28|oscar davidson
+125242096|1216672.55|katie young
+-1137321242|4114451.92|alice white
+-1386573581|-2000786.39|oscar miller
+1861912179|3110263.83|fred carson
+1257207573|-745890.65|gabriella young
+1866021510|-2965674.46|nick van buren
+904541370|227009.34|priscilla johnson
+-973286257|1039642.52|quinn ichabod
+-632851543|3753442.21|katie underhill
+-277521740|3560509.15|xavier quirinius
+-394316057|-834648.77|jessica white
+-972473152|-3423358.99|tom steinbeck
+-221169795|2357646.69|nick robinson
+910387824|4430686.15|oscar king
+-427874045|-4613077.71|victor polk
+2003518493|-4734447.59|tom underhill
+1283102012|-3778194.02|sarah ellison
+825369779|137166.23|irene brown
+-1706849983|490375.72|quinn falkner
+1697547381|-3907219.59|rachel robinson
+1191720915|4285023.66|holly quirinius
+-1180661246|403305.88|wendy underhill
+1243630593|3990965.71|zach falkner
+987159434|-1804888.36|irene brown
+1308146714|-1888139.82|zach miller
+240497737|3290751.51|quinn ellison
+1723191456|-3736601.91|fred van buren
+1416903311|-2276784.08|luke ellison
+1944294340|-2129406.56|calvin laertes
+2037702912|-2363947.82|irene johnson
+1505971585|2681170.6|sarah carson
+-892677662|-3594685.03|mike ichabod
+-150151801|917436.31|victor polk
+1282346365|4501158.35|xavier young
+971112282|-2925384.56|ulysses thompson
+282105096|-1943531.9|fred carson
+-1388798325|1113902.09|zach thompson
+-814628458|-851985.78|yuri quirinius
+-145709118|-525622.29|zach quirinius
+1971671792|3527161.01|ulysses allen
+700778773|-3149566.55|ethan davidson
+-1802705797|-4066432.49|xavier davidson
+1367484149|234176.4|oscar miller
+1030413923|2023429.33|rachel carson
+-1476608905|4808591.58|tom laertes
+1453307856|3657514.11|holly van buren
+867777776|1296166.6|fred garcia
+-1243675958|220929.05|mike carson
+1638345265|1885680.62|yuri polk
+189748496|-2060829.56|luke garcia
+1272888749|4177567.2|yuri ovid
+-896450268|4717261.32|nick garcia
+1885244524|142549.71|david thompson
+-1810413278|2778975.24|ulysses xylophone
+-1456552778|1665703.64|rachel ichabod
+-1501392570|-3080704.57|irene miller
+-1000334949|-4075889.82|quinn xylophone
+-550557132|3669969.19|holly laertes
+-1684753423|-2998743.57|bob nixon
+-1498569900|-2037208.85|ethan garcia
+648307046|-156779.9|fred ichabod
+-1010416672|-4470362.44|sarah underhill
+-1613428309|-3858932.93|sarah xylophone
+-119382658|-553373.36|alice falkner
+-615317790|-4393836.45|ethan robinson
+1211676028|-3144068.25|bob carson
+-482911773|-1499148.27|rachel hernandez
+1170706942|88469.49|zach garcia
+-296350737|2779876.87|yuri laertes
+-1163944252|-3624937.48|victor johnson
+1652684506|2206872.91|katie davidson
+-1919276717|-453385.1|wendy ovid
+-886111762|-2261008.8|alice polk
+1589130917|-4986515.22|nick falkner
+923014363|-3499962.51|bob johnson
+1013005494|-4003466.65|gabriella falkner
+-1267207257|2960058.51|fred quirinius
+-707532691|-2768176.16|ulysses king
+-1335724727|-3454209.03|xavier ellison
+2016074980|211749.64|rachel young
+1676758135|3420149.65|xavier falkner
+-1869229295|-4987471.34|tom thompson
+371057894|1862078.22|david nixon
+-759332592|4207093.45|oscar nixon
+1658683585|-429753.43|jessica allen
+-319797540|1472733.09|jessica xylophone
+-1751508630|-3917194.02|priscilla allen
+473848348|-493053.16|yuri ovid
+16341904|4747138.67|quinn quirinius
+1465851199|1024863.27|mike ellison
+403094082|2366687.98|alice king
+-706935261|-1930555.51|priscilla nixon
+-1613179510|-4888734.81|priscilla miller
+855770870|-955597.58|oscar carson
+1010016423|-174155.46|luke hernandez
+786522148|-3684013.97|victor garcia
+-434790119|-2503737.42|gabriella miller
+591146075|-3589884.46|david carson
+-833840413|3613581.63|jessica nixon
+62941197|-211025.02|ethan hernandez
+1523439577|3766768.15|yuri king
+-518210552|-3691100.29|tom garcia
+502808286|-3651346.06|quinn steinbeck
+-1168015635|-2886766.96|david underhill
+-1817283257|4335940.25|nick zipper
+961430447|1065963.46|ethan brown
+899717103|-4719960.44|alice johnson
+1086157680|-1689369.03|ulysses xylophone
+527338147|1918570.79|calvin davidson
+1793126204|2160704.8|rachel xylophone
+1459862496|-3228415.16|zach underhill
+-809690458|4474454.59|yuri ovid
+1903515686|4008728.74|ethan quirinius
+-2043278804|4177653.92|rachel laertes
+1154323266|3413333.03|priscilla nixon
+1274376431|-4570033.3|holly van buren
+2057746281|-2814525.24|fred king
+-605987032|3347128.23|tom van buren
+204169459|-4706804.68|zach van buren
+1423146823|-4410640.94|victor quirinius
+742294292|2472966.57|irene ovid
+532632036|-852484.86|tom falkner
+-1287334674|3979872.88|yuri xylophone
+-382041930|-1229607.4|ulysses xylophone
+-1518540857|1512624.14|calvin miller
+-754659521|-3265192.63|ethan davidson
+-544360879|-1289640.77|alice hernandez
+1522729997|2524825.81|david thompson
+954911299|2239082.83|ethan davidson
+1659924369|2768367.51|oscar quirinius
+-197758797|-378539.85|yuri nixon
+1861354989|-4529689.9|david ichabod
+728674020|-2323677.85|sarah robinson
+1598187354|4516847.78|katie young
+-1528754010|2936827.73|bob laertes
+-1675411721|-4588386.48|luke van buren
+461884987|-3932740.76|sarah ovid
+1174086158|4425439.77|nick van buren
+-1637123191|485032.45|yuri brown
+1455563337|-3433600.15|ulysses ovid
+1527661317|-2905371.74|victor laertes
+-950623430|44630.71|holly thompson
+-564728173|-1710041.64|alice laertes
+789137476|-4911333.6|wendy ichabod
+1460436308|812272.8|jessica zipper
+-1748170276|3269600.52|holly hernandez
+207098139|1095815.92|nick thompson
+-669499010|-860591.01|ethan thompson
+15165180|46946.2|irene young
+72274430|-2904738.93|fred falkner
+-1473632722|-3737408.45|luke laertes
+453250558|3037964.74|sarah ovid
+1196942654|-3492012.85|oscar nixon
+-2126663524|1900418.34|yuri ovid
+-350541493|-1465191.94|holly allen
+-1277774653|1306360.45|david nixon
+905028531|-2693796.16|holly robinson
+1829316014|3950082.8|katie carson
+-1339170176|-1368179.82|ethan hernandez
+1602140016|3679195.01|xavier steinbeck
+-1671932804|4778101.77|priscilla ovid
+-1374150504|4130766.78|bob quirinius
+-1302184534|2660032.21|bob white
+817323606|4098186.03|ethan davidson
+-334114393|4775116.76|katie xylophone
+-1090039116|-4338357.6|katie ovid
+-1510772473|-4921336.47|luke ovid
+188160168|-2732657.7|victor miller
+-1165964318|-658591.7|ulysses davidson
+243141691|1875393.66|oscar steinbeck
+-1704542464|-3222979.62|calvin steinbeck
+-667849478|4154362.65|yuri brown
+1552740964|-6582.22|david robinson
+1462878043|3325933.71|bob ellison
+928833332|-1997577.68|wendy young
+1370895865|747636.44|quinn underhill
+-369967076|-251055.73|sarah robinson
+593659285|2114077.23|holly brown
+-1736746296|-2982503.45|rachel ellison
+-873453712|1996209.65|nick laertes
+-1927390700|1345825.31|holly nixon
+191447896|-1946289.1|irene laertes
+749889517|2910757.47|jessica king
+267966329|-1028889.07|luke zipper
+-83743960|-2452925.09|sarah johnson
+1125795291|-774305.13|alice zipper
+903811249|726248.11|victor ovid
+-1943484412|2408082.81|sarah ellison
+-897373972|-2549085.69|oscar johnson
+-1408927936|-350890.79|david robinson
+-1903120117|-348699.46|katie underhill
+-306828154|-339637.86|luke white
+-604377640|1804674.49|xavier young
+893355237|772733.19|tom miller
+-1211361247|-1615310.86|quinn robinson
+-1084397567|-3839220.84|oscar thompson
+930419895|-4451056.17|nick van buren
+-1029860340|-4244233.59|gabriella king
+-280635172|3822869.72|yuri ellison
+1022525242|2438924.35|irene underhill
+-1991868077|-1625780.85|calvin nixon
+-278437714|-3963528.61|alice brown
+184063404|15634.21|ethan falkner
+-61146311|-4045955.28|ulysses polk
+1764655705|3732086.1|nick ellison
+-1447435622|-3579909.1|quinn underhill
+-126659680|-1035869.37|rachel miller
+-439617540|-4900995.78|gabriella garcia
+-965937005|4375743.89|mike king
+510055636|-3738727.02|oscar thompson
+939466911|-4382682.29|nick miller
+-1082107008|4035155.19|victor xylophone
+2070015813|2573392.56|luke nixon
+-1756644563|1648275.63|tom hernandez
+1024104715|-1435193.11|xavier van buren
+279933168|-918853.54|nick young
+729043430|-3800743.01|alice nixon
+-2111487458|3662124.03|fred thompson
+-781903749|-34291.23|sarah robinson
+-1801690065|4468118.18|katie zipper
+250522371|-2888777.07|alice garcia
+-1442179862|-4401889.88|holly ellison
+117088715|2579311.25|xavier quirinius
+-580691579|875730.67|gabriella white
+1202133104|-4184654.29|victor ellison
+-197491213|2782759.3|victor king
+-818623888|-4897312.23|holly hernandez
+384900515|-516876.77|nick xylophone
+1205395632|233753.98|wendy xylophone
+2014356826|548595.85|katie hernandez
+1730426106|-4055066.79|ethan carson
+1234610660|-3269972.65|jessica garcia
+-405580378|-3634652.2|victor ichabod
+1003921884|2762420.6|irene ichabod
+-1486788888|-1006333.99|victor steinbeck
+-1188431681|-3271555.66|alice thompson
+407564411|3283575.39|alice garcia
+56594412|-3270176.47|calvin underhill
+896206250|-3383514.69|oscar steinbeck
+1977738884|4655869.17|jessica davidson
+-210840877|3871603.96|irene ovid
+-1817119985|2477495.03|nick xylophone
+627754747|-3808254.34|luke johnson
+1330846308|2577077.68|victor hernandez
+-140109525|-834729.43|priscilla xylophone
+1461298547|472524.02|calvin ellison
+-302407582|-2173424.74|irene young
+-1134021731|281821.85|rachel polk
+-783963500|-2535052.97|gabriella falkner
+16579033|2369983.68|oscar hernandez
+-1896414722|-4113467.21|katie garcia
+354399997|-4092024.71|ulysses johnson
+-1914146410|-2610728.88|tom thompson
+1721709475|2951157.43|ulysses ichabod
+1879976088|3665437.29|alice ovid
+-2004453210|1580067.05|katie allen
+-644273571|-4229275.76|holly davidson
+1285730614|-2942045.25|wendy miller
+106418926|4444231.97|ethan carson
+1661828599|-3013186.14|ulysses ellison
+-588369367|-676836.05|mike young
+367223959|-4953641.59|mike garcia
+-1847204291|-1353478.74|luke falkner
+-1824615457|1922037.78|priscilla young
+148177907|2026289.62|bob ovid
+-1707575741|1543837.85|fred thompson
+-684601848|3461666.31|gabriella allen
+1966282051|-2786184.57|jessica nixon
+173297251|-4552978.88|fred king
+-1734726067|542008.63|holly nixon
+1151181615|683804.0|wendy ellison
+340565451|1278174.91|luke king
+-348074253|-4877955.93|jessica xylophone
+981237691|994619.65|sarah carson
+1824220723|460382.86|david van buren
+-1899518762|-2413784.43|tom steinbeck
+503471707|4622212.67|david polk
+-1810050523|-839.53|rachel johnson
+-1575167801|2854052.74|oscar garcia
+-1676395340|1676001.72|priscilla zipper
+-1756112312|-1935258.45|nick nixon
+1900799366|-336295.92|priscilla brown
+474595985|-2361981.63|quinn thompson
+1706127911|-3259447.95|holly hernandez
+-761145287|-1716446.45|tom white
+-1507660697|1213489.27|ethan van buren
+-1235112629|2452036.41|mike johnson
+-1496053727|-4732958.72|victor king
+-2121183818|-644079.96|fred nixon
+2035628075|-1442596.76|ethan young
+-1306434570|4572616.65|ethan carson
+430541749|3395025.67|victor robinson
+-158819912|52526.42|jessica van buren
+548279226|-835095.55|katie miller
+616808825|-3831054.16|tom zipper
+-1915945886|1526929.61|victor robinson
+34773843|-3540042.98|wendy young
+436512853|3919728.67|tom van buren
+-910124441|3441603.73|holly white
+180520063|2904762.75|zach ovid
+-723730007|3544712.58|luke brown
+-238679887|-1192596.96|priscilla thompson
+-251408891|-2265950.12|nick thompson
+1196162569|4224406.03|quinn carson
+-1993981685|3905389.42|victor quirinius
+-1503150507|-482644.66|zach zipper
+1847184031|170643.32|gabriella underhill
+-1419755635|3472887.07|jessica robinson
+-1342711440|-1892546.04|bob allen
+-874285767|-2075797.82|tom brown
+-1287778030|-943120.16|luke ichabod
+1791192430|-1156310.3|irene thompson
+-1113007590|1619778.87|ulysses ellison
+906404253|-4947665.8|holly miller
+1232307088|1968610.08|luke carson
+-370226331|4327887.01|priscilla thompson
+2135903877|3874159.82|alice robinson
+-1171833814|4587640.4|victor xylophone
+1427475384|-2207999.62|sarah robinson
+711117357|-2868920.13|luke thompson
+352223379|1842543.7|victor carson
+1525739590|813331.96|bob davidson
+-807115811|933434.34|irene miller
+115756435|-4117158.12|david miller
+-462930954|3859562.33|bob van buren
+1505515708|4515290.51|wendy garcia
+632164232|3320690.5|sarah nixon
+1742366371|-2639554.55|holly robinson
+-506298658|-4608180.42|oscar steinbeck
+-2105792882|4896880.16|bob xylophone
+1252052826|4829587.13|priscilla white
+-1679163860|534649.62|wendy allen
+-1550023709|1845615.76|oscar nixon
+-134066671|-2196819.85|nick ellison
+-1723130877|-596375.8|jessica polk
+-1237767307|-3226402.6|katie ichabod
+1543721266|180551.51|wendy miller
+-1871604143|3988318.46|wendy brown
+2106619030|-258509.64|ethan polk
+1091580736|2341475.94|holly quirinius
+-558417665|520624.91|holly van buren
+-748804441|2143969.01|gabriella king
+1755941915|-4794094.49|xavier zipper
+-15570169|-3549634.21|rachel hernandez
+-1050609886|-3316339.52|fred underhill
+514453141|1787820.52|quinn underhill
+-874405295|-4901847.73|katie allen
+-1650361235|4334223.32|ulysses underhill
+-659662829|4734578.7|ulysses ichabod
+-1294990362|702385.08|alice davidson
+-76934211|-2969421.33|irene steinbeck
+-1655061243|-1400468.56|alice miller
+1680053516|-3447731.52|bob robinson
+492065538|700093.52|quinn robinson
+1133538554|-457155.76|calvin brown
+-908276149|3672896.4|yuri underhill
+721408379|-178444.31|holly underhill
+-156549738|-2677824.71|sarah zipper
+404917571|918999.73|mike underhill
+346233310|1935590.21|tom nixon
+1228957097|1752606.65|rachel underhill
+1516700515|2377046.0|gabriella ellison
+-753425347|-3712349.66|gabriella hernandez
+-1215382018|-2748681.84|irene davidson
+242737833|4715458.84|holly garcia
+722407781|-321721.06|holly young
+-1142155518|1286221.25|sarah xylophone
+397708869|2417655.89|katie laertes
+-570229884|2476371.48|alice polk
+-1191734486|1110921.45|nick carson
+532781169|-1053763.07|victor allen
+-544617302|3683729.84|yuri johnson
+-1436106847|4048227.42|gabriella ellison
+50192999|2364885.62|luke johnson
+-1165754179|2956807.71|wendy king
+1436065457|4175636.31|sarah xylophone
+1234227832|2625934.92|luke zipper
+-1686589208|921098.33|yuri ichabod
+-28397525|3379205.0|tom brown
+-1303526664|1874385.21|irene ovid
+1183550585|2570064.77|oscar carson
+2033256593|-3275507.58|sarah quirinius
+1599441196|3351264.73|wendy ellison
+-2137111842|-1064855.28|zach nixon
+-197316293|1921954.81|wendy polk
+907771348|2460960.35|nick laertes
+1927794088|-2185096.27|ulysses ovid
+1114381459|1576328.37|zach thompson
+-1120229663|-930889.06|irene davidson
+-1576587203|2475028.13|oscar steinbeck
+-1224578175|-1854469.2|xavier hernandez
+922034504|1676437.2|nick ovid
+198400720|3612572.56|luke falkner
+684881230|-408055.94|jessica van buren
+1485997156|2424599.25|nick thompson
+1333232559|-2511878.04|quinn ellison
+-1534771305|-4506619.87|wendy steinbeck
+-2144646744|-2118434.39|holly zipper
+1883258073|-2514254.18|holly king
+-522988875|-4376005.33|alice falkner
+289214484|-3617474.21|gabriella laertes
+-491527083|-2369926.49|yuri miller
+1671241352|634408.36|bob ellison
+1082940543|-4482628.79|calvin falkner
+1449673710|190673.4|ethan van buren
+-338556812|346096.3|nick polk
+1646303691|787776.6|holly johnson
+1012282793|962196.27|alice white
+-651468490|-1632855.84|oscar ellison
+-356912885|-4134004.46|yuri ovid
+498627189|-2252577.56|katie robinson
+-1111465794|-2142533.03|alice xylophone
+1229494098|2082567.68|fred johnson
+1387000614|-2146705.73|calvin robinson
+-1122360350|-3714255.29|xavier allen
+729928177|-1091140.97|victor johnson
+-559427787|170337.28|mike quirinius
+1008611895|-4007268.3|ulysses nixon
+-1112176577|-2072037.57|jessica zipper
+1879526180|2479049.32|rachel underhill
+188914877|-4823527.27|luke davidson
+-991296853|3712150.05|jessica ellison
+-1150791911|-3183931.9|wendy carson
+1830702107|-4047753.14|oscar ovid
+1404288361|2467324.17|fred laertes
+1859049160|-4453933.61|jessica brown
+1478299433|-1734078.47|jessica ichabod
+-10841352|2530271.33|alice xylophone
+-1408036799|95138.52|alice miller
+1777521101|2472807.73|xavier quirinius
+1285029729|-4166706.81|yuri polk
+894429434|2168480.02|priscilla polk
+22819582|2141994.55|zach zipper
+1723555148|1370693.76|yuri van buren
+-652827852|-1507332.29|quinn thompson
+-1355663493|1970020.55|oscar johnson
+1923035263|1265885.85|alice zipper
+363224257|3774892.14|calvin young
+-1219037661|-2648739.95|sarah quirinius
+522330405|2585104.31|yuri white
+-974656399|-1034567.47|bob zipper
+-1091118216|2470949.03|ulysses garcia
+-2091523444|-3039006.19|oscar laertes
+-107177097|-4474165.65|tom xylophone
+1684158515|-4937776.2|luke young
+1296978076|3198065.58|rachel allen
+-2082730060|387908.66|ethan davidson
+-1401928585|1542907.39|tom white
+1426439443|2566274.19|zach zipper
+401462689|4388153.01|david ellison
+1392726804|-2269881.24|ulysses white
+1099133111|3773322.73|irene xylophone
+1715873705|-4471591.73|zach young
+-1718031656|4937507.19|ethan van buren
+813584611|2564300.04|luke falkner
+2111450509|-3068102.13|bob white
+-887013234|-3124788.92|victor robinson
+-634542521|-1532698.15|calvin nixon
+-1630070810|12758.8|katie young
+619426703|-2223509.01|ethan xylophone
+-1750213056|342886.43|ulysses steinbeck
+-1992529711|316460.86|priscilla ellison
+961594483|-1231808.36|quinn polk
+-1733098162|4038690.57|david ovid
+-1438393256|-3654907.74|david garcia
+-512002681|-965963.04|tom van buren
+-1529121077|-155196.51|zach hernandez
+540980818|2308432.32|xavier brown
+-105479187|389462.4|holly underhill
+1176569674|-1225049.14|alice ellison
+-623302504|-760228.17|bob ovid
+1964512860|-2458629.96|yuri king
+-198700733|-4945281.95|luke laertes
+-566706010|-401466.61|tom zipper
+1535978946|4629146.77|sarah ichabod
+334118106|2189438.6|fred ellison
+2113290935|1705410.44|oscar nixon
+355439082|4038217.27|fred falkner
+-1355322245|-2928075.71|gabriella davidson
+714977446|4924232.0|nick falkner
+1248140526|-3701440.46|ulysses xylophone
+223572363|-4263852.12|alice steinbeck
+-481322878|4183738.95|xavier brown
+-706369532|2734656.08|calvin laertes
+960385084|-1336347.83|oscar van buren
+-2027781445|-2329623.15|calvin allen
+-732693062|-2060341.62|priscilla garcia
+-432005927|-4741369.22|zach thompson
+1255730002|-3553721.76|fred van buren
+826246722|-2734628.88|holly brown
+1332389358|-2910058.74|fred falkner
+1223307566|2695990.33|victor zipper
+-1339668691|2021505.26|david davidson
+-2030587480|-4841732.35|victor ellison
+-1326783002|4883107.38|zach white
+-528048152|3907489.07|priscilla carson
+-1011460038|2725061.05|holly king
+2025343101|-4248168.34|gabriella laertes
+822648228|-1509219.96|quinn quirinius
+2070689215|-3586956.78|calvin ovid
+73846988|4931401.45|zach van buren
+-790588410|-3952611.03|calvin johnson
+-1383483388|-1834237.85|wendy king
+1325829768|-3740145.98|nick king
+-673333054|134426.35|ulysses thompson
+977736973|-4787197.72|rachel king
+1508488563|-4970286.64|fred zipper
+279155361|1155442.9|katie ellison
+860462707|4159362.58|alice carson
+-1295053815|-533698.4|katie ovid
+1305890318|4614373.21|zach allen
+1219784887|2088212.24|rachel thompson
+-1050286882|-2052702.45|victor allen
+500020076|4179862.5|david garcia
+-1346144544|4349038.59|katie davidson
+-1372516416|-4155994.47|luke miller
+-1766542577|2594913.73|bob king
+1174194251|-372194.02|alice johnson
+95704689|2419220.07|mike miller
+1219573043|-3454895.79|wendy ichabod
+-1921312557|1182781.05|bob white
+2006072628|-4001925.72|sarah allen
+-1534987106|483084.9|yuri carson
+1496243083|-1519253.95|jessica ovid
+1939274910|2661503.31|nick ellison
+-1760873652|2584669.44|bob ichabod
+-316690031|-3866787.94|katie miller
+-410946305|4176906.71|priscilla polk
+-235223363|-2944773.55|jessica van buren
+982160807|1398230.91|ethan young
+-85929956|2584850.25|ethan quirinius
+-1782966661|1697833.08|gabriella laertes
+-1849934408|-1550609.34|wendy ellison
+1889137387|-4355373.41|jessica falkner
+349282110|2555550.79|wendy carson
+-2032351519|-1368519.9|oscar white
+980500488|710787.06|fred zipper
+31813927|2974442.22|fred thompson
+-367915273|2715758.42|nick van buren
+-1136431484|-1146838.68|bob van buren
+-1571928319|-2706648.0|priscilla robinson
+-1301153808|4413830.08|katie young
+-1257849785|1320449.61|ulysses xylophone
+142811875|2972705.29|gabriella young
+1063247608|-1443160.36|mike carson
+1141828796|-4767477.44|calvin thompson
+1986576516|-1093273.28|priscilla young
+1345637029|-2944707.09|rachel carson
+-1685487431|-894553.23|priscilla miller
+1302535555|-560117.83|ulysses hernandez
+-1090793498|4333671.63|priscilla ellison
+1019631062|3688763.49|xavier robinson
+1743878181|1171263.06|nick robinson
+-877742699|-3654649.12|holly polk
+-264408245|2364751.12|ethan king
+867829344|-1606902.54|priscilla king
+694862893|4048476.3|yuri steinbeck
+607279284|4865030.46|oscar van buren
+-2131269299|-4290587.23|luke ichabod
+-2118810419|-4407660.14|bob king
+-612164617|1147715.69|wendy polk
+358465660|-1340424.58|david nixon
+-41780602|2552730.89|ethan zipper
+1555742009|-3524501.82|holly carson
+-463698618|-3901341.91|nick brown
+691435507|-4622935.36|katie miller
+200690976|-1442677.27|zach carson
+270057628|4859760.34|fred robinson
+1168582700|4289692.66|xavier allen
+722571626|4343738.9|calvin ichabod
+1572733859|911270.49|victor robinson
+793978425|2154310.29|bob miller
+673795063|1696429.04|mike allen
+880898711|-4940096.22|oscar falkner
+-514804996|-3145778.43|oscar nixon
+-1347770015|3442399.04|mike ellison
+-460805799|-4689715.99|calvin laertes
+142283509|351102.72|bob miller
+-1151337526|3219712.29|jessica polk
+-631636115|8235.4|zach garcia
+649690577|-977818.02|fred steinbeck
+187975770|2497817.11|katie ovid
+502158059|-292664.0|alice laertes
+4071929|7521.78|zach garcia
+114625633|1626196.07|alice johnson
+2111241688|3091152.44|david quirinius
+1487521101|2009668.97|bob falkner
+46238052|-4191345.88|zach white
+727873394|4459130.24|alice davidson
+618361049|676674.35|oscar zipper
+-539016463|1575644.47|xavier van buren
+834540424|427059.36|zach polk
+-1550704638|953738.42|katie miller
+-387067786|-2138030.73|victor xylophone
+966638907|4047840.39|luke young
+-1293839690|4767937.08|quinn carson
+-187894256|-1481614.23|david underhill
+-1292772585|-2147425.38|mike white
+40871041|-4536813.79|mike davidson
+-382385931|4631632.44|wendy laertes
+1401316935|221119.92|irene underhill
+-163490225|1191355.98|quinn laertes
+-1750298413|3695915.25|nick white
+-487025798|888572.58|luke garcia
+-1799827215|3901726.77|sarah miller
+1726512376|-3377343.77|priscilla robinson
+-2079579642|-3267045.3|irene zipper
+759966312|-3616522.29|ethan polk
+-1457932779|4805537.58|gabriella xylophone
+-291127411|-4232191.48|rachel xylophone
+-283745529|933873.28|priscilla garcia
+141678796|-813512.47|jessica falkner
+1713636081|3465421.72|tom hernandez
+1628467911|2934842.67|bob davidson
+-124961699|3012261.89|victor falkner
+-1997864747|-1804246.25|jessica brown
+-1328272223|-1714817.23|alice underhill
+-540784516|-2386872.4|victor steinbeck
+-785006082|2999019.72|victor steinbeck
+-373374896|4954516.11|calvin steinbeck
+1671103492|-4376133.07|luke ovid
+-1727680845|4712733.38|sarah young
+1371498582|670225.0|katie nixon
+518891881|-4762967.7|bob brown
+1239510773|3765429.96|xavier davidson
+809857290|3556668.06|priscilla johnson
+-634753899|946599.15|fred van buren
+2094072743|-2437951.02|xavier ovid
+2025353900|-805314.28|ulysses white
+-294111155|1148800.85|mike king
+121502776|545293.37|victor davidson
+773312800|-391390.41|david ovid
+54410096|-1604886.25|xavier polk
+-358562738|3916926.43|ulysses underhill
+-1281759269|-3021148.58|quinn steinbeck
+-695987203|554473.75|quinn ovid
+1394216321|-1162672.31|mike garcia
+-411960311|-1466491.21|yuri xylophone
+1039592764|4985498.43|wendy garcia
+-2135486413|-131400.16|zach allen
+1064717125|2932940.54|victor falkner
+1003499516|579370.8|calvin johnson
+-243538797|-4882111.81|david quirinius
+696485245|-4978904.14|holly underhill
+-1202651982|-3924902.66|fred laertes
+-277402940|-3260080.7|sarah ellison
+-17998575|4836375.64|sarah white
+2007047583|100601.75|wendy quirinius
+-1910995974|-1844278.11|gabriella van buren
+1298459635|4380119.6|priscilla steinbeck
+-785877835|-3286215.86|quinn hernandez
+-2088989990|-657193.29|priscilla underhill
+2140367739|-584068.12|wendy underhill
+-1323357894|2172809.99|jessica hernandez
+-737085611|-1228727.43|ethan johnson
+-864283990|-1813943.7|victor quirinius
+1866954984|-480742.88|zach garcia
+-994315853|-2136720.38|sarah nixon
+1981681898|3382792.3|alice zipper
+1604878533|-4489155.47|rachel falkner
+-13274699|300138.6|irene davidson
+-503050355|-1881724.79|ulysses zipper
+1094721310|-1693976.66|oscar ichabod
+-880886183|4238242.92|rachel laertes
+319569087|789333.59|yuri ovid
+-883948578|75456.3|bob king
+724921876|3261609.16|rachel zipper
+1174309243|225571.11|fred allen
+120540484|-496307.05|gabriella falkner
+-1893287741|1994238.54|calvin thompson
+-2033521543|-25265.25|nick van buren
+-1592325897|-2607811.18|nick steinbeck
+458326786|-3645546.64|priscilla garcia
+-1771616836|-905847.32|holly quirinius
+1916347079|1942130.13|yuri polk
+-2036713416|-2060734.0|zach falkner
+1664543656|1213589.05|nick king
+-1507499450|1004257.11|ethan xylophone
+-368248353|1465695.57|priscilla thompson
+1814325138|2318594.42|yuri white
+-189984378|613535.8|nick allen
+-291872070|-4542523.62|oscar nixon
+1192697032|-2882847.94|nick xylophone
+-1322955589|2899517.62|tom polk
+1479721840|-732875.21|ulysses laertes
+501406563|3136450.96|calvin polk
+1744044483|1026185.73|rachel polk
+297033599|-4058793.74|holly white
+-10649099|1200986.4|alice xylophone
+972244454|-4429222.8|ulysses brown
+-282990956|-4179004.98|wendy miller
+-1336770870|-1920130.8|ethan thompson
+-1576272447|2009894.71|fred thompson
+-1082374961|-673655.63|xavier ovid
+881413131|-857772.14|ulysses laertes
+1399790036|-114103.61|calvin nixon
+2132955981|868162.31|xavier polk
+94462326|-1488458.96|holly ovid
+-132662054|-4809968.94|david johnson
+1907841453|651955.2|yuri quirinius
+539357136|-1122345.97|katie nixon
+648765468|4916858.61|luke zipper
+-1043076207|-1615910.62|mike carson
+1713506028|-4823964.11|irene falkner
+500452195|584406.28|ulysses quirinius
+-25065918|2437068.95|jessica ovid
+-1377471492|1896623.3|alice thompson
+-1164761165|-3641557.76|tom quirinius
+-1610554257|3145108.17|quinn underhill
+519270214|4300649.3|calvin zipper
+933874592|1380608.86|jessica nixon
+974557669|2612577.87|oscar van buren
+-1176229796|3852460.72|yuri ichabod
+-1309193403|-1979826.89|alice ellison
+1230712210|1825994.61|victor xylophone
+978382250|-1211333.29|priscilla davidson
+248395129|1310072.47|victor king
+154575915|1819434.01|nick falkner
+1296403835|-1854814.17|priscilla nixon
+-752688092|-4861406.32|ethan miller
+1059430960|-647785.2|sarah brown
+1389757725|-4003674.88|oscar steinbeck
+1833491944|169060.34|sarah king
+332559977|4138939.13|xavier robinson
+-872696472|294337.62|nick thompson
+1938956299|4621909.57|yuri ovid
+1421293015|2423539.34|victor van buren
+1923266764|-2713165.82|mike white
+1703722033|-4166622.41|alice brown
+408374673|4001394.34|calvin xylophone
+850960766|2895275.47|gabriella xylophone
+-153981366|-2442936.27|holly davidson
+-537106241|-957033.06|ethan thompson
+-2086763761|-753688.91|holly nixon
+-8017313|2368105.56|ethan allen
+-238665651|1545215.75|jessica ellison
+1724081792|-328188.66|bob white
+-1621288184|4412318.01|ethan nixon
+138207879|3548062.1|luke miller
+2062257830|3625699.61|victor nixon
+2077527964|1191642.03|alice ovid
+1098058685|3195618.2|david young
+-216596691|3666210.84|victor miller
+2128942032|2148959.31|fred hernandez
+146632595|1685899.27|ethan xylophone
+8218636|-126252.7|ulysses hernandez
+59264617|4471398.88|zach robinson
+-1106060958|931768.17|xavier van buren
+1596915505|247064.3|katie xylophone
+940131974|-4287133.07|david young
+-1430940587|3258922.04|ethan allen
+-54314802|3419438.42|sarah polk
+-1062156874|-4321536.95|ulysses nixon
+261450448|-3398083.02|irene laertes
+-138044606|-1070330.6|yuri young
+1351803999|-1000810.59|priscilla nixon
+-1811821758|-1811689.78|calvin ichabod
+1038044475|-1659021.14|alice van buren
+1682441050|-1015258.45|nick white
+-2030685063|2318620.59|holly laertes
+790104196|-3273614.19|fred carson
+1944214693|-2597310.63|alice carson
+-1369105057|123882.46|victor robinson
+953562377|-4870191.92|david zipper
+1476935589|2547428.91|mike steinbeck
+-1896442221|-4334027.62|wendy ovid
+-334373703|4867080.25|victor king
+450375502|3374314.48|wendy davidson
+-895128973|2484236.92|luke ellison
+-102903418|-3376073.67|xavier brown
+55979203|4835892.63|xavier robinson
+1368304587|-1097551.48|wendy falkner
+228075977|-4134538.78|sarah king
+-1349691996|-105626.24|yuri miller
+185808219|3336706.94|nick quirinius
+414656652|-1444736.74|mike hernandez
+1028383302|-4300164.35|rachel underhill
+-1436835956|-2016953.72|nick zipper
+1625447079|3348265.06|jessica falkner
+1963458455|3222615.3|luke carson
+-1499958738|4945450.19|irene quirinius
+-718247568|-394900.81|calvin steinbeck
+584909861|4440138.79|gabriella laertes
+1026439605|-4963547.05|david thompson
+-1144738661|758831.37|luke johnson
+-727230857|3368826.35|rachel davidson
+715159286|3390944.27|calvin carson
+-793556637|2895434.99|oscar ovid
+765396454|-4463096.69|zach davidson
+-771498922|3143031.82|mike xylophone
+552619883|-1967797.37|nick laertes
+-690558446|93832.46|victor van buren
+2100763996|1382018.41|jessica van buren
+1402947371|-3687787.33|yuri king
+305489360|4156902.27|yuri carson
+1547280292|4712012.25|calvin garcia
+-629403618|986557.64|yuri young
+-1268780130|685971.05|xavier underhill
+1795041911|-1914237.15|jessica robinson
+1489015338|4603970.14|gabriella garcia
+1843970384|593244.67|quinn white
+-401925325|-2145894.65|bob ellison
+332973319|-3092543.24|victor nixon
+-945569317|-3005548.23|priscilla young
+-2135391346|978189.48|nick underhill
+2065747696|-3981415.5|zach falkner
+-1264680186|-1426123.04|calvin miller
+-27537584|4798000.1|mike garcia
+1159450506|-2242586.69|zach king
+-1934885832|-4991212.66|calvin steinbeck
+-204250717|1353966.11|priscilla nixon
+1147458565|1437375.65|holly polk
+533591047|-1240398.99|david polk
+1699125212|-513553.93|quinn johnson
+-699154744|-2110594.22|nick underhill
+29886227|-3969460.82|katie quirinius
+1266805495|857217.77|fred ellison
+717507464|2929869.91|priscilla robinson
+-1989313184|1856123.24|xavier xylophone
+1431710250|-4086156.03|victor white
+-300040208|2217806.63|priscilla zipper
+-2117844653|-2519803.18|xavier miller
+-1125046044|3464370.77|fred king
+-374277426|2373084.06|yuri white
+1558787027|-4766109.16|bob xylophone
+269805979|1231330.41|katie robinson
+-285023188|4518086.16|yuri van buren
+1374823378|3681164.31|calvin davidson
+1626180030|-4186167.09|nick polk
+983226154|1114318.25|fred ovid
+1606037362|136116.49|mike white
+-1339397351|1827725.8|tom davidson
+84519218|2761389.11|quinn carson
+-108130936|2699420.96|luke young
+-768421461|-4507801.37|tom xylophone
+-2124671094|2127154.22|xavier quirinius
+1961820544|-3273564.0|tom johnson
+-1533059684|-2483038.92|holly miller
+1523609858|4879601.98|fred zipper
+-130561197|-1119152.67|mike miller
+77126266|-2144924.99|luke carson
+1882303624|-1148069.78|bob brown
+-1704876363|2216069.95|gabriella garcia
+-1576841630|2839998.8|holly young
+-736559529|-420568.45|irene zipper
+1669939397|1336725.3|ethan johnson
+1388747282|-3215089.54|tom carson
+871236895|3977797.68|yuri king
+285862116|-1164901.23|fred falkner
+-1746570881|1459483.2|zach laertes
+-285053228|2912298.59|ethan nixon
+1525616495|4364923.47|nick young
+1725425651|-159777.5|rachel quirinius
+-265858005|1699240.78|zach steinbeck
+-1519512933|4606332.28|zach hernandez
+-2124592153|3298364.51|mike king
+1050663168|4971323.86|holly ellison
+1781679212|2239267.19|wendy robinson
+1381905864|-4528860.14|jessica hernandez
+1946270538|-471519.63|fred nixon
+1742434337|-3622499.11|ulysses falkner
+81355310|910850.55|katie white
+590707587|4844141.67|ethan laertes
+548470864|1949195.0|bob polk
+968373297|1030779.59|holly quirinius
+-1456512516|4464775.36|holly steinbeck
+147795205|4814933.75|david ellison
+-893323374|666324.84|rachel falkner
+856136763|-1246009.77|irene underhill
+-492024930|-4291347.76|quinn king
+-1683096534|1142453.69|ethan brown
+-1171020849|-636856.84|zach zipper
+1098119723|2330265.52|mike robinson
+-1592327432|205862.24|david hernandez
+1604690227|-1990875.86|nick ellison
+-915346842|-1517063.46|rachel xylophone
+-1051779405|4605485.97|xavier van buren
+129414947|-204393.86|nick polk
+-284336161|796312.99|victor underhill
+925162513|-2693150.48|nick carson
+-1155210629|4165872.68|xavier king
+289575887|-4678391.26|nick robinson
+-1708343380|-1261783.09|fred thompson
+-1366260246|3601817.95|xavier miller
+343736459|-3079658.16|sarah davidson
+-845917995|-2223628.59|jessica ovid
+365101928|-3641567.81|sarah xylophone
+-1265662317|-1553312.36|rachel laertes
+858058236|47827.52|yuri ovid
+1334606871|1174197.38|jessica laertes
+1451568127|-2693960.48|ulysses hernandez
+855958470|4132281.07|ethan brown
+-1087137058|183940.72|nick davidson
+1025350052|-4576014.69|david ellison
+645239424|-279237.42|zach garcia
+328064010|-3090460.14|irene laertes
+-1258403739|3413204.82|ethan polk
+-346720760|-2684798.43|sarah miller
+525034864|-951819.5|ulysses hernandez
+360820907|-2215988.71|nick young
+916906069|-1809420.8|irene ichabod
+851068653|-540546.67|nick robinson
+-771779244|2618346.24|ethan brown
+-559307782|1698692.67|rachel laertes
+-1844872776|-2579303.28|jessica white
+-1079282180|3116395.78|priscilla miller
+440401016|2960502.99|quinn ellison
+-1040406584|3203175.58|tom ichabod
+-1260915019|4850585.76|mike ovid
+-1259589137|1715771.03|xavier ellison
+339791363|1262663.74|zach underhill
+-1471776733|-813529.53|calvin ichabod
+-219682863|-2949234.65|fred ellison
+-2041568779|-1524603.25|irene ichabod
+1466406975|3451141.9|ethan polk
+-1271259567|2283920.06|wendy xylophone
+592977998|2533382.78|calvin young
+485545800|-2315148.08|gabriella quirinius
+-136681946|3758957.38|quinn ichabod
+-1438377972|2204054.45|quinn van buren
+-310335521|-2958852.54|fred falkner
+-1751328492|4322148.92|priscilla zipper
+794348584|1414410.07|yuri underhill
+1421909757|3226680.51|bob king
+-807864948|387593.81|bob garcia
+1737975426|-1989711.42|irene quirinius
+1110174786|-2903224.24|gabriella hernandez
+1262931071|485261.56|oscar carson
+1315282915|-1818180.13|xavier hernandez
+-1786623196|2685470.32|ethan falkner
+1655553347|-3696445.81|katie king
+-406743666|3069807.05|fred van buren
+-1928335768|-2332812.6|wendy van buren
+-1297522299|3490827.28|nick carson
+-1406721703|-4174749.35|ulysses davidson
+1112791821|394886.48|zach zipper
+-775489482|3175175.27|wendy polk
+1243717646|3549578.39|yuri carson
+1204787162|-1454682.72|gabriella falkner
+-1245377982|-1686417.22|ulysses king
+1538472947|4368040.14|gabriella king
+-1853633506|-401110.32|david davidson
+-204433830|-2965616.76|xavier falkner
+1571817864|983537.85|sarah ellison
+760929534|1636298.45|ethan steinbeck
+79507300|1638718.33|katie brown
+1200739559|1492779.63|sarah zipper
+-567906979|3085793.99|tom davidson
+-929697323|3442191.71|victor thompson
+-2118194304|-4604571.67|rachel johnson
+-1853679191|-4039117.83|yuri underhill
+1982738823|-1540272.04|quinn steinbeck
+165646270|1864873.86|tom robinson
+972998617|613719.62|zach king
+877875098|4153966.17|tom steinbeck
+1135637159|-3934412.26|holly thompson
+-701923390|-4492544.51|xavier robinson
+1594004928|4036855.43|rachel steinbeck
+860615841|3421861.37|sarah quirinius
+-732175386|2156303.35|holly johnson
+-534918969|4764395.7|ethan davidson
+517048006|4947509.49|alice brown
+-1432671529|162223.61|tom xylophone
+-1142555406|87430.96|zach nixon
+-1997898999|-1692060.98|holly brown
+1130697049|-2178261.91|mike robinson
+1341520144|351971.56|zach king
+1094327361|1992131.88|ethan xylophone
+-1701746498|2715992.71|david falkner
+1598898085|967315.34|irene polk
+-1519941964|4870386.42|ethan carson
+432413065|1989562.87|zach polk
+125745160|-793580.91|mike young
+-940407638|-85594.23|priscilla nixon
+714694327|-4020695.29|victor white
+678429821|291289.78|sarah laertes
+-501319138|3333320.23|fred underhill
+-177114215|-1790508.86|zach ellison
+-1012660740|1054266.35|rachel nixon
+-719778874|4851411.27|fred underhill
+-1333087306|-297593.14|bob johnson
+389739889|-2490430.83|bob van buren
+2126934219|4622676.43|bob polk
+-1775908799|-379179.35|irene steinbeck
+427328181|4831612.8|rachel johnson
+2108545249|328605.97|holly young
+1736319918|4502015.63|priscilla polk
+-275486594|282020.72|fred white
+-1853600486|-3629870.79|alice brown
+-1174087592|-2733497.14|jessica van buren
+-996078730|1727656.92|gabriella zipper
+1995798646|3044466.15|quinn hernandez
+1075979558|-3761868.86|nick thompson
+964063848|1335636.05|oscar polk
+-611493758|3744430.04|jessica thompson
+1365524878|4876619.35|david brown
+1258868432|2698542.86|yuri underhill
+1954689125|3343296.89|wendy laertes
+-1954354860|2580986.71|mike young
+1419771596|3863596.5|luke carson
+1126896232|1852878.11|jessica young
+979449658|-2089195.33|gabriella thompson
+84217945|3347273.11|nick white
+652697945|2351028.65|nick garcia
+-1468249974|1090888.72|jessica young
+-257540470|1301135.95|mike xylophone
+-788872969|-2847221.0|rachel ovid
+-697032392|-1472103.06|quinn thompson
+-1491973307|-4762401.24|mike robinson
+-225752568|1944817.56|bob ichabod
+-580489261|-2459781.22|fred thompson
+-777218492|1193699.16|victor underhill
+33044618|3744184.39|fred thompson
+1350156565|-2474482.27|bob polk
+2034119268|1713176.17|zach king
+-1869245074|-1052805.66|ethan carson
+-210587173|3567720.15|luke ichabod
+-1589395293|-692055.56|luke nixon
+-110630466|498025.33|fred ovid
+-1090580938|-3057267.23|zach allen
+-389053140|-4720556.75|priscilla carson
+428589887|3625775.13|ethan nixon
+627344534|2254893.47|alice underhill
+697443432|-1077863.51|nick ovid
+-564139237|-1573972.55|ulysses young
+-986055805|1129600.49|ulysses white
+-972512100|1318068.85|irene ichabod
+-292568548|-1599669.39|ethan young
+-1607612840|-1168700.43|jessica carson
+954006688|-1473779.66|oscar van buren
+-1828703494|-3674651.32|yuri nixon
+-1846354455|3835264.46|nick miller
+-493252530|4048513.25|zach young
+-408056506|2591784.76|david van buren
+-536498138|-4345619.52|ethan brown
+604631400|-1760316.3|xavier johnson
+2090327824|831109.28|tom steinbeck
+618905782|-2348318.42|mike ovid
+2012039918|-1368157.92|calvin underhill
+-172532751|3105346.88|luke robinson
+-1325748083|-1348022.28|zach ichabod
+-1235373525|1397935.43|oscar robinson
+-47772985|2071846.91|tom falkner
+-1499419249|539401.47|victor polk
+1985210090|-2069513.06|jessica robinson
+1636204132|-1262843.17|victor davidson
+1818925388|1730516.68|luke hernandez
+870913606|1206672.62|gabriella robinson
+-892624314|-1220402.05|nick xylophone
+2083936148|3674465.32|nick miller
+1243248413|-524750.4|gabriella young
+300119432|798151.54|rachel underhill
+67003277|2412629.91|nick johnson
+-534065296|-2644238.31|luke steinbeck
+-321673252|-3755405.6|quinn nixon
+78956021|-4708672.9|holly nixon
+-2015615812|594825.29|katie underhill
+-1694977139|-904556.23|ulysses quirinius
+-489545346|2446333.48|wendy robinson
+325574994|-2062737.71|ulysses ichabod
+-1278029257|26671.54|nick polk
+-466151422|2261101.71|bob robinson
+-879654182|-4620350.65|ulysses ovid
+-1967662899|-4373694.98|wendy brown
+-661925499|-4264389.84|mike johnson
+-1302500289|-508405.63|alice quirinius
+647828847|3988111.82|gabriella young
+18242925|2691617.63|gabriella johnson
+-210927306|-3544304.44|nick young
+983873064|1079652.75|bob davidson
+-1827111304|-3986554.97|quinn steinbeck
+-663751457|-831359.25|victor brown
+958848523|2710094.62|bob brown
+-1553051783|-3887018.83|luke davidson
+477070125|29129.3|oscar white
+-64489888|2619251.87|bob ovid
+118274414|-4408065.29|alice carson
+-1316961961|765686.04|quinn white
+1460163675|3335128.09|victor falkner
+-151352317|-3349434.31|tom young
+-1992479473|-2128446.69|xavier allen
+-1726637519|60459.01|irene nixon
+-418327257|-2918578.8|gabriella miller
+-719113169|727630.51|bob ovid
+1519859673|4443067.23|wendy davidson
+-763799071|4545727.48|priscilla garcia
+1254469917|-2106663.38|victor white
+932347731|-466970.08|sarah robinson
+-824594399|2754504.24|fred young
+-1323249666|-1815833.29|tom laertes
+1814402431|3674422.03|zach brown
+493225450|2419680.68|mike quirinius
+401471236|-4168584.79|tom robinson
+-237664126|-2168505.69|luke carson
+589437388|2985046.66|priscilla nixon
+226087600|-4890246.28|ulysses johnson
+1522538790|-3014448.64|ulysses quirinius
+-885710180|2968045.21|irene king
+-867490928|-2396907.31|luke davidson
+-1022706397|4105213.74|wendy garcia
+1196378254|1073239.87|calvin quirinius
+-1451905577|-230295.0|gabriella thompson
+-806825622|-3423253.49|ulysses falkner
+1513975867|-4094697.36|gabriella ovid
+1209457627|-1898338.38|holly laertes
+1702062442|-422434.5|wendy young
+498809184|4632050.84|wendy nixon
+1010178412|-3596540.2|mike young
+-1345241427|538219.07|wendy garcia
+-709197342|4724111.45|sarah carson
+2036762216|-4603206.46|ethan ichabod
+-92210212|3309507.85|quinn falkner
+-282886321|1415219.79|luke carson
+1221175534|-3656445.21|sarah thompson
+103629106|88504.68|fred young
+-1491311647|-4619987.12|mike thompson
+926508454|389583.17|nick falkner
+544252809|-950444.14|victor white
+-448671691|-527425.72|wendy young
+2109781692|-4873118.4|jessica hernandez
+-1744879774|4559344.84|bob steinbeck
+422586958|-3511788.02|rachel king
+1407941369|2755306.27|zach brown
+1548458651|-3283437.09|david van buren
+-1581789862|-1106724.14|fred polk
+337557527|3019807.56|calvin ovid
+1966767274|3208354.05|yuri hernandez
+1203204812|2393506.04|calvin zipper
+-2023096757|2528592.43|david falkner
+-1471143208|-1129892.98|mike ichabod
+-1856900643|641455.64|wendy king
+1091087422|104369.29|tom allen
+1263231669|-4540102.51|fred quirinius
+-1739318598|-4289611.65|sarah steinbeck
+-643453540|4109467.76|yuri garcia
+-430533057|2425201.32|rachel steinbeck
+1696695827|-100691.25|yuri polk
+726788984|3933161.64|calvin davidson
+-1053998665|560903.07|wendy steinbeck
+-1345469585|4482508.73|ethan garcia
+-975857345|46078.58|luke van buren
+-1781130239|2014547.88|rachel steinbeck
+1354548813|2455769.7|tom ovid
+729689864|2609607.33|zach carson
+-127722979|-2529458.71|bob steinbeck
+785434693|-2109687.78|zach nixon
+-38851432|-2008487.65|ulysses king
+221749844|-990491.87|holly davidson
+807625869|-307677.7|victor ellison
+1472844131|-4214996.06|victor garcia
+-485152519|4612025.06|bob hernandez
+-1780279704|2941141.58|wendy falkner
+1283283094|810553.04|bob robinson
+1446133751|3455758.49|yuri ovid
+-1762721724|414563.22|yuri allen
+401608221|-137948.48|david laertes
+-1687060017|2812244.98|gabriella zipper
+221803708|1346015.41|priscilla davidson
+2094449739|-764807.76|calvin miller
+-925817767|4721623.59|nick van buren
+2090363912|-1874372.83|quinn steinbeck
+-1646328983|-839594.93|victor laertes
+-1993618512|-3748812.69|david allen
+-1302115718|-902971.82|katie falkner
+-298708670|373534.61|alice ovid
+-1802742839|-2690191.02|jessica quirinius
+176648422|-601689.96|priscilla miller
+1052209950|-1789315.51|priscilla van buren
+-1424323379|4486989.68|tom nixon
+-98867942|1443305.93|luke steinbeck
+-1236900726|2615646.98|irene ichabod
+1296999710|-1106732.95|wendy young
+1679392700|-3676162.41|oscar hernandez
+2028228521|1758333.68|zach king
+-545186079|4597969.94|victor davidson
+111981249|3308908.72|nick underhill
+105829009|-1241681.59|rachel brown
+-964861777|4307624.85|priscilla thompson
+959003823|-1603617.36|wendy xylophone
+980816196|2506880.89|oscar robinson
+347652133|-2484824.98|ethan nixon
+-1084494746|-3584229.51|alice laertes
+-1553411262|4747118.82|mike garcia
+962169996|4592982.57|calvin laertes
+825513432|-1439552.7|tom steinbeck
+-1942080912|362828.34|alice ovid
+1830029236|4654269.16|xavier zipper
+118412072|2229221.59|irene garcia
+-886385413|-4031619.94|rachel underhill
+-794431384|-560822.63|rachel allen
+-751012473|3577203.64|holly robinson
+-436791667|-4804047.32|victor johnson
+-191127174|-2708484.75|katie underhill
+803569351|4545530.3|gabriella miller
+-570347423|1993538.17|victor van buren
+-1918026868|4529026.18|ulysses king
+-224679476|3594438.76|gabriella van buren
+1905759227|1894103.32|rachel johnson
+-1983025885|1383434.66|irene allen
+-2075175146|3137025.62|wendy ichabod
+-1040681954|-4410050.01|fred garcia
+-1727264313|-2280388.92|calvin nixon
+1251679791|4414338.29|victor underhill
+-1211927757|3390411.4|alice white
+1823358140|1452507.32|bob ovid
+1535262933|1858957.92|bob polk
+-1027731717|-3255790.07|xavier johnson
+2095962175|-1245832.11|yuri steinbeck
+1317917123|-2315682.59|david young
+1630797472|1701936.59|fred brown
+729238224|1196921.08|xavier steinbeck
+-419473817|4625952.49|irene hernandez
+-804775929|2274769.27|rachel brown
+-1686511223|2416376.27|priscilla miller
+-1571300633|3730508.92|victor van buren
+351161187|2879093.6|gabriella thompson
+1658479934|818208.01|sarah underhill
+150259502|43206.25|luke robinson
+-695392346|2595956.17|katie thompson
+-1223335819|-3350839.73|rachel steinbeck
+-829940874|-3673813.06|david steinbeck
+1843773713|-4458227.98|katie underhill
+2077264181|-4806670.81|jessica zipper
+-15254367|2318055.56|luke polk
+-412405323|3369972.31|holly underhill
+-1661439124|114161.79|gabriella miller
+1360974663|1539494.65|zach nixon
+952543020|-3166751.38|fred underhill
+1542362602|-3307408.18|zach king
+-772146611|3088724.69|bob young
+1456728103|3977005.42|sarah king
+3069530|2063183.2|zach ichabod
+1031443486|4406257.91|luke white
+-1769426697|1705591.82|ulysses van buren
+898211585|2130615.22|quinn brown
+-282073499|-2836837.5|fred ovid
+-1627607335|803449.42|nick davidson
+-1896384644|-1302199.82|mike thompson
+-1805088396|2353515.71|gabriella miller
+2025305272|2457939.36|gabriella carson
+251761868|4214941.49|wendy laertes
+497635988|-4553729.51|alice zipper
+-1112251308|-3459196.35|quinn garcia
+-1196450955|-1594585.16|zach hernandez
+-1068425270|396024.69|tom johnson
+-615480891|519856.94|katie king
+176504768|-4392354.15|victor xylophone
+1115932964|4276595.55|luke falkner
+-148797215|1857189.57|yuri falkner
+-1526831186|876171.52|alice nixon
+101821038|4862631.21|nick van buren
+1693528249|-1524641.07|alice xylophone
+1267219928|2215564.61|victor johnson
+1141891249|4146843.0|gabriella king
+2095934510|-1474852.3|luke garcia
+617303500|-4648545.99|calvin davidson
+-1324612654|1960022.63|alice van buren
+570155980|-1016998.1|quinn miller
+-127165507|-2344686.2|katie davidson
+166410517|-1925899.34|quinn thompson
+275075894|-3478494.63|mike underhill
+-652179022|-1917457.59|david ovid
+-1054204978|3545793.61|oscar zipper
+766888229|-1007127.5|xavier quirinius
+-523298583|-1160102.37|gabriella young
+-434317789|-2581696.9|katie miller
+15715159|-1729751.17|mike ellison
+836680555|4997333.5|jessica polk
+1124812967|-67397.95|mike brown
+22273940|462682.75|ulysses ellison
+448132386|-3459505.66|yuri laertes
+-2071781018|-3004023.79|wendy zipper
+111930564|1604464.77|irene nixon
+1478504706|-1544929.91|holly laertes
+-1154430320|1584642.78|luke johnson
+-1130000858|-2214043.02|ethan young
+59416325|3294269.34|yuri king
+-1354464751|-4451731.6|holly king
+1941042600|556413.58|luke davidson
+-1010647032|-1485730.86|luke davidson
+-1025665320|-876205.13|david laertes
+2122916196|32042.4|yuri carson
+672370279|-2500142.37|ethan ichabod
+-958545232|-406582.28|ulysses garcia
+1984608442|4089312.54|bob ovid
+-25973439|1767957.48|luke quirinius
+96010058|-553722.27|tom laertes
+-1622751926|504185.52|victor steinbeck
+1022776040|-1305985.76|fred falkner
+1994454628|-1049476.23|nick underhill
+-1193368296|1477120.48|quinn underhill
+432021756|4287289.93|rachel quirinius
+806646599|-3561381.24|david johnson
+-2042681053|-225246.56|bob thompson
+-844614164|2057845.49|rachel falkner
+-2050929745|-4194050.14|oscar johnson
+1194918546|-3164575.01|alice nixon
+-1753704254|2447790.74|oscar brown
+-1897841782|486755.79|sarah laertes
+-1901088001|-1117309.19|david young
+-373026621|-48731.52|david king
+-292697832|-890764.56|luke underhill
+-999150366|3606766.96|zach zipper
+305773438|1224942.51|david thompson
+-1282081086|4973828.64|luke zipper
+329749089|-33146.84|fred falkner
+-1167592737|1038115.69|holly garcia
+1909481521|-577296.01|david johnson
+954757458|1583918.91|luke garcia
+-2007991450|-4373995.33|katie garcia
+1314055283|3990828.16|victor ellison
+2110271044|214095.54|xavier davidson
+1160395066|-3252087.7|luke polk
+-1933138485|975533.1|jessica garcia
+116753896|-3564100.77|xavier ovid
+1194498319|1376062.66|priscilla allen
+1809161395|2257200.54|luke ovid
+-634001225|-1158022.83|irene nixon
+-1594189982|3125841.93|luke ichabod
+-700128690|2931671.94|zach ellison
+-1233408493|-2632803.4|tom hernandez
+73659817|-2841468.59|irene brown
+-34259280|3074567.03|victor van buren
+1717703322|514921.82|holly laertes
+-1594844233|2252259.45|david allen
+942580632|-1195393.53|tom garcia
+142956635|893212.26|calvin johnson
+-1965441490|560109.47|victor robinson
+1907762484|-1185814.19|oscar allen
+644791372|-1785645.33|calvin johnson
+1012743151|436073.64|gabriella king
+-1469738536|-3934372.55|xavier xylophone
+-1036344895|-3626531.89|ethan carson
+-1700796418|-4044927.24|ethan young
+-229277817|3986464.43|ethan steinbeck
+-475798467|820438.18|zach polk
+-709803042|-2258999.36|rachel ichabod
+-598174933|-165321.92|david hernandez
+49710742|-3502844.64|alice van buren
+-461519024|4692968.45|irene quirinius
+929138248|1513718.4|alice young
+6482618|4775928.27|fred davidson
+2082924540|3828799.05|yuri johnson
+903042579|-2396890.71|victor falkner
+932245025|-4345084.72|yuri hernandez
+-1939191207|226923.17|victor thompson
+205584200|-519073.22|holly young
+808532623|1583475.22|irene ellison
+-505704868|-4403398.98|holly ichabod
+-1259423477|2764641.84|wendy zipper
+-185533108|-1194006.11|victor nixon
+435595281|-440099.37|priscilla polk
+1043134566|3418860.69|tom young
+1388990483|-2238467.76|quinn laertes
+774433757|-1696444.59|irene garcia
+-948746840|3535907.13|calvin ovid
+1339328026|-2104833.97|yuri king
+-632641525|463938.57|gabriella falkner
+1148033267|318742.57|ulysses robinson
+-1772043044|362892.65|david xylophone
+-1180358425|4212096.62|bob young
+-470636484|-3016375.45|fred zipper
+553962720|-2329701.33|jessica laertes
+703865005|-1452851.09|oscar king
+66202169|-3560149.92|oscar robinson
+-1800973250|-4093216.28|sarah brown
+-1923436565|-4394214.58|yuri johnson
+332841173|-2864663.57|nick carson
+-1228437199|902895.84|calvin ichabod
+-1567329323|3324422.58|yuri steinbeck
+-258972911|-433796.09|luke thompson
+1525153081|-3100685.54|luke laertes
+959832448|-1371453.65|oscar nixon
+733753608|-2101686.91|xavier robinson
+-1781739493|923952.82|alice zipper
+250854828|3295563.92|katie carson
+1127155621|-4720775.74|calvin hernandez
+-987297658|-1618960.55|david robinson
+517733601|865287.1|luke garcia
+1221800425|208934.45|david johnson
+-1417558452|-3760316.13|victor underhill
+1682417717|-2941034.72|ethan johnson
+-1876398624|-3506492.49|gabriella zipper
+-856629809|-2271081.88|sarah young
+608200957|-2234958.34|holly falkner
+832608283|-2396077.85|irene king
+1550148601|-3051978.11|jessica hernandez
+858355164|-2708317.67|fred polk
+-1501102067|-4175426.36|jessica king
+978307442|1492524.27|sarah ellison
+-963116659|-373062.55|wendy allen
+-423671825|3814009.62|mike quirinius
+-1469347390|-2881012.08|sarah laertes
+-132749306|639448.01|bob johnson
+-338648394|3570486.1|yuri thompson
+1808673848|-973828.59|holly xylophone
+-299668282|-81830.64|tom steinbeck
+50432464|-4680265.98|victor brown
+1779758309|4656617.02|victor young
+1101150851|-621566.36|nick allen
+-1219039279|168860.76|calvin falkner
+256022743|1420875.12|ulysses ellison
+1391892900|3769528.55|victor underhill
+314818470|-1332065.96|ulysses robinson
+-59467118|3657781.49|zach davidson
+-749956281|3132599.75|calvin carson
+1566138532|-2338480.63|sarah zipper
+146222919|2145409.93|bob underhill
+1524345821|1879431.24|nick laertes
+674069962|384050.51|fred garcia
+-566270710|3316834.57|david miller
+1654132179|-730228.47|katie davidson
+-1826221797|-4762826.24|ethan davidson
+-694574033|1676443.82|gabriella van buren
+1129355766|565428.85|oscar white
+1863978683|4656636.23|jessica ellison
+670534381|-4017216.02|katie steinbeck
+410551352|4904983.1|oscar ichabod
+-1233218980|-2849215.77|yuri hernandez
+-732175632|-3992757.75|victor johnson
+-1228870877|-2851138.08|oscar allen
+121208517|1756237.25|ulysses garcia
+1608290301|2938867.85|luke steinbeck
+1855157151|-1982026.2|irene king
+874223580|-3980490.07|quinn king
+2097720099|-3785408.7|quinn miller
+-590928441|318512.97|jessica ovid
+-902593231|-1711867.15|holly ichabod
+-603799554|-1619674.74|sarah allen
+2020239534|-1509319.13|victor underhill
+-117049008|784217.86|fred johnson
+1526660732|-4886588.25|xavier ellison
+-1602425640|-4054280.26|holly king
+-787576146|1641638.64|david thompson
+1257074602|251599.99|calvin brown
+-1882304433|4542336.13|fred thompson
+1484905747|-4369768.02|yuri ichabod
+403847789|-3398036.52|priscilla xylophone
+-571649676|-281719.35|wendy young
+-19416802|1169085.31|victor ellison
+978179297|-1545013.26|tom polk
+1202318238|68073.58|fred hernandez
+-1163087334|2677325.18|mike white
+1481859335|1543636.86|xavier brown
+153407329|4198669.96|gabriella allen
+2089484014|-568922.67|sarah steinbeck
+-2012043671|-4711209.51|jessica laertes
+160991759|2722950.78|alice ichabod
+1006976336|-2154030.93|ethan allen
+1084681956|-4048306.88|zach davidson
+816223835|1133451.3|david falkner
+1905348762|7485.31|holly robinson
+-1707918088|-3519234.54|victor nixon
+-115334402|-4191975.6|rachel falkner
+-221628895|-4328845.45|nick underhill
+-455165912|-4832951.47|rachel falkner
+-518359216|2635600.9|mike zipper
+856868108|3179252.48|bob laertes
+1538421355|-91996.62|jessica ichabod
+746520653|-961083.93|sarah young
+-301592967|-2044037.4|irene polk
+-770034481|-2216799.84|fred laertes
+-1007428608|1688963.68|mike quirinius
+-1988329989|4078113.21|david hernandez
+987449692|1174383.8|sarah johnson
+1655043532|4358846.89|alice white
+607280761|2266533.33|irene johnson
+-77892374|3772348.06|alice thompson
+-929240928|-2784372.91|sarah ichabod
+-8965258|4540694.49|sarah polk
+-1203024288|-4013418.08|katie zipper
+-1444303515|-3461381.39|fred ovid
+-1975365429|1264449.4|quinn steinbeck
+-1488832645|3943593.98|sarah xylophone
+1784945257|57873.1|quinn miller
+-33897924|3250079.68|gabriella quirinius
+664729015|-2090471.33|tom white
+2074963564|3566202.68|priscilla laertes
+1961307149|-2745611.62|oscar white
+2076205828|-4425827.32|alice nixon
+1341371198|2212705.42|quinn garcia
+-134097259|4799348.88|rachel nixon
+-303840431|-2794604.69|wendy thompson
+-1525807468|-771128.77|bob laertes
+-2020802783|-3482853.22|bob falkner
+-1575553773|-668307.55|irene quirinius
+-423831092|-3332989.28|alice brown
+-1774990915|3694506.42|alice steinbeck
+1220943923|4870517.81|calvin carson
+1301720377|-2512612.89|katie white
+-1412015875|-1805683.7|holly garcia
+1031042631|1184084.71|katie ichabod
+-936716592|562635.19|zach ichabod
+-1629252055|2385926.25|ulysses laertes
+943665102|2142155.45|katie robinson
+-1079054795|324128.34|nick garcia
+-1434574215|2018022.08|luke polk
+-906581331|-371306.73|rachel carson
+-1401810068|3456591.54|yuri laertes
+350451503|-1701570.51|ulysses johnson
+1801851084|-2707328.58|tom polk
+1756327765|-2195609.16|ethan xylophone
+476968768|1662405.48|jessica quirinius
+554279510|2645044.42|fred carson
+698466735|-2054702.11|irene quirinius
+-443290921|-750302.96|bob white
+1004177488|-3466530.33|jessica ichabod
+-2123117526|-4322559.86|wendy polk
+-1958736715|2924307.85|wendy ellison
+706481313|1454139.77|nick garcia
+-1108592706|-275378.83|wendy quirinius
+687960273|47771.62|mike van buren
+-1635093553|2264281.27|ethan brown
+-1398907453|980000.79|nick ellison
+-444161899|3061786.44|rachel polk
+1709297812|-573608.76|xavier quirinius
+478236471|-2654668.28|xavier king
+1745491964|-4531492.86|katie brown
+133387480|991346.83|rachel thompson
+1994694808|-1001798.93|tom ellison
+-2061803872|-1145570.31|holly steinbeck
+-653459678|-2619153.59|tom nixon
+1017535952|-1016404.14|alice white
+-728841256|3553431.58|sarah robinson
+-449041510|-1387468.05|alice polk
+-1890477897|703604.22|fred allen
+1043921813|2415890.01|zach young
+1738360929|-2323304.76|nick allen
+264366402|-3598204.32|nick white
+-798188089|-471227.94|jessica king
+-2025951273|4919751.3|priscilla miller
+865150169|931298.89|bob quirinius
+-144392909|-4256338.44|yuri quirinius
+-350829818|3762963.81|alice allen
+-266477586|-502493.12|gabriella laertes
+1552577065|-1767868.56|oscar miller
+463904168|2235305.5|david laertes
+1051865897|1262112.56|zach garcia
+-990930391|-4623435.0|ulysses king
+-1134420600|755053.89|rachel allen
+-2130431759|-2863225.45|victor underhill
+1394169733|-571349.97|nick brown
+1119570512|3895392.14|sarah davidson
+-2016063826|-1082567.77|david steinbeck
+20208152|-3904320.3|sarah brown
+831441638|218700.51|fred underhill
+1168107901|-4629310.0|victor brown
+1643511380|-3924149.21|holly ichabod
+1556963027|-4523992.57|alice thompson
+664572594|-3973395.25|priscilla garcia
+1420129565|-1610061.82|victor ellison
+-164595470|-1265548.61|bob garcia
+22794923|-4908426.68|ethan xylophone
+973273661|-4048672.0|xavier brown
+393984321|3160884.14|xavier thompson
+273533363|-2929160.56|holly young
+-15158417|-1804234.49|tom garcia
+1858448494|1380954.58|sarah robinson
+494451241|1126993.13|ulysses ellison
+-1305523780|3514912.21|zach hernandez
+-315006253|-4595845.93|david robinson
+875554706|1003232.32|priscilla brown
+-16479980|-3933668.53|gabriella king
+-657469538|-1273127.49|wendy ovid
+613247338|-4524334.67|david laertes
+495904639|576455.76|calvin ellison
+-1568174015|4215613.63|tom brown
+966998807|4600422.96|rachel ichabod
+-1694290226|295477.12|calvin ovid
+-408690195|4859071.75|holly robinson
+1864606279|-3170288.02|nick van buren
+-151693040|-4429230.82|sarah falkner
+150868740|-4673525.29|quinn ellison
+-995282672|3750207.39|irene thompson
+-1455466338|2072073.46|sarah xylophone
+296573906|-541577.31|oscar xylophone
+-932866763|-2342343.73|xavier carson
+-1130329991|3157561.04|priscilla young
+309759332|649226.13|bob underhill
+-1457822836|971464.79|oscar young
+-432114803|-2397317.19|rachel davidson
+413740|2573216.34|irene falkner
+924449906|2412654.85|nick white
+309449473|-2317761.66|nick falkner
+800321648|3708971.57|zach van buren
+1159234053|-4539860.98|david ovid
+-364161114|-2142703.53|xavier nixon
+-1585156971|4369454.4|xavier allen
+200845542|-4259776.79|calvin underhill
+630564168|-2040649.45|fred miller
+775051877|-2504984.87|jessica hernandez
+127628153|-4127737.38|gabriella nixon
+-762075071|2499693.97|gabriella ovid
+1382204551|4215744.14|katie white
+-754800517|4264684.01|gabriella nixon
+1406750004|-3443484.16|david johnson
+180856075|-3331736.53|fred robinson
+-958563911|-1146329.46|ethan underhill
+751928335|-4907313.48|irene van buren
+2132631382|881589.94|jessica quirinius
+-771255496|3634548.73|alice allen
+-1025413993|-4655023.37|sarah underhill
+1624090374|417272.82|luke underhill
+-1682496135|833776.42|gabriella carson
+773650311|-4318134.5|ethan robinson
+-1505566919|2704081.79|sarah laertes
+-2054501506|-2240466.89|alice quirinius
+333495159|2833275.93|fred ovid
+-1727944645|3050379.21|victor davidson
+488252654|-2912370.74|wendy quirinius
+-2097303647|2428791.94|ethan garcia
+689461233|-3481234.7|ulysses king
+-642259205|4087711.49|luke quirinius
+-940433564|2409704.24|david steinbeck
+-1396924238|-4799245.79|gabriella nixon
+776762110|-2822285.77|fred king
+-1545535482|3806714.07|gabriella johnson
+1878924784|-1584228.16|xavier garcia
+794548565|656259.23|victor xylophone
+-945384141|852171.74|katie johnson
+1353964186|3147980.75|wendy allen
+1143585597|2166307.8|yuri ovid
+-678674655|-1985711.02|fred laertes
+416615123|-1142547.64|holly van buren
+-863885658|-4346419.26|luke johnson
+-899705570|1410277.45|holly nixon
+-2109846685|-1724052.64|nick robinson
+-163317928|-4127392.45|calvin robinson
+-176942162|-1752146.2|sarah miller
+1997142699|-4306383.69|calvin ichabod
+-1078842572|1701243.22|priscilla polk
+1661109536|1383096.5|katie johnson
+-1211646722|3635049.45|mike underhill
+-304943967|3531881.96|alice polk
+1129407165|-203811.89|sarah johnson
+1549348218|-1717249.69|zach laertes
+854767377|2300010.15|yuri white
+407309303|4831235.18|priscilla thompson
+-1901232183|-2333775.76|alice garcia
+-509979435|-702216.75|jessica thompson
+-826185148|-2831368.01|holly ovid
+-1418112316|-4311428.99|calvin xylophone
+-1706383476|-290653.8|holly underhill
+2058794813|2722821.74|irene hernandez
+-113695049|-2825122.8|oscar ichabod
+-1420053680|4429411.29|jessica davidson
+-634123343|2938235.85|wendy robinson
+651911458|4119024.49|gabriella xylophone
+316087682|1909800.54|priscilla quirinius
+-1686963849|409292.26|holly nixon
+-235171159|3838810.09|wendy allen
+2115448902|2197434.21|jessica nixon
+134720289|-3591721.78|victor ovid
+-1759868347|837786.68|ethan xylophone
+1526021937|2540050.94|david young
+301603073|1395559.68|ulysses ichabod
+429825816|-2732467.74|holly van buren
+-1711874609|1279693.37|alice xylophone
+-1820712154|477862.4|quinn davidson
+1502653372|3669416.01|jessica laertes
+737307881|1588517.99|fred young
+163303246|-3890239.05|fred brown
+-1643779707|-2181255.36|jessica ichabod
+290100908|3177.95|alice white
+-1841204679|2680926.06|oscar ellison
+-1488052350|78789.73|zach underhill
+1716863939|-4854191.63|holly garcia
+1199775693|2039304.41|priscilla ovid
+326745659|-1337164.74|wendy white
+-1664760506|-1707969.99|victor king
+1619707478|622336.97|sarah robinson
+-423177438|4227300.49|mike hernandez
+-476510616|-4301421.5|priscilla brown
+1915807853|1776168.62|ethan davidson
+2114650235|1702756.49|bob quirinius
+2144461097|1871773.63|nick falkner
+2061743192|-1071458.21|xavier white
+1919787751|-141677.63|irene carson
+-1938302134|-3032740.87|katie allen
+719251876|4541441.96|holly ichabod
+-1217971967|4521546.54|xavier nixon
+-163388399|-292459.84|nick hernandez
+-1338602503|3667960.91|fred polk
+1517694598|3310342.95|zach ichabod
+517804307|2687058.68|fred zipper
+1146032726|1473306.98|luke young
+-327506574|4788623.61|ulysses ovid
+1841392443|4289497.66|bob carson
+-1538185549|-3022634.68|ulysses xylophone
+850216720|3842147.98|david nixon
+1512838425|2877650.83|gabriella young
+901303325|-3360711.8|alice thompson
+791116886|1439450.71|ulysses young
+-1852292966|-3162900.1|wendy miller
+-1536878885|-1499956.43|gabriella white
+-1282817485|-3283868.67|yuri falkner
+-1652404993|-3113270.06|wendy ellison
+252995250|4088756.03|calvin young
+-1074238349|-1903211.95|wendy nixon
+1335930665|-1093241.77|katie zipper
+-606401123|-2910049.38|tom nixon
+-922190566|-3637249.78|katie brown
+1253008383|-1587987.21|wendy miller
+469935510|-36633.34|gabriella brown
+1545468860|-2533909.35|alice polk
+258400357|-1942369.02|mike garcia
+167410121|3865208.88|quinn thompson
+1382332781|-4084324.2|xavier davidson
+285287798|3899355.68|bob brown
+-245405403|2878538.6|ethan nixon
+652648786|-4432376.91|alice steinbeck
+-756376387|-3237186.35|alice brown
+-1679485289|-221448.3|tom steinbeck
+2034520374|-1395857.29|gabriella davidson
+-645103204|-287877.1|zach white
+-1678867283|3719080.75|zach ellison
+-2054252790|893069.27|calvin young
+-249907348|1423875.18|david zipper
+-1742851911|2251661.58|victor steinbeck
+-370634054|601789.07|alice white
+-1045027190|2302104.52|holly davidson
+133799259|825308.62|gabriella zipper
+-1200407912|1913956.13|david ovid
+1220296240|-4475848.79|rachel ichabod
+-335339123|-921048.69|gabriella zipper
+748714885|-2477963.42|nick miller
+-38142195|-4686104.07|oscar quirinius
+2040429034|112

<TRUNCATED>